Refactor Register*ListenerReply classes
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / test / java / org / opendaylight / controller / cluster / datastore / IntegrationTestKit.java
1 /*
2  * Copyright (c) 2015 Brocade Communications Systems, Inc. and others.  All rights reserved.
3  *
4  * This program and the accompanying materials are made available under the
5  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
6  * and is available at http://www.eclipse.org/legal/epl-v10.html
7  */
8 package org.opendaylight.controller.cluster.datastore;
9
10 import static org.junit.Assert.assertEquals;
11 import static org.junit.Assert.assertNotNull;
12 import static org.junit.Assert.fail;
13
14 import akka.actor.ActorRef;
15 import akka.actor.ActorSystem;
16 import akka.cluster.Cluster;
17 import akka.cluster.ClusterEvent.CurrentClusterState;
18 import akka.cluster.Member;
19 import akka.cluster.MemberStatus;
20 import com.google.common.base.Optional;
21 import com.google.common.base.Stopwatch;
22 import com.google.common.collect.Sets;
23 import com.google.common.util.concurrent.ListenableFuture;
24 import com.google.common.util.concurrent.Uninterruptibles;
25 import java.lang.reflect.Constructor;
26 import java.util.Set;
27 import java.util.concurrent.Callable;
28 import java.util.concurrent.TimeUnit;
29 import java.util.function.Consumer;
30 import org.mockito.Mockito;
31 import org.opendaylight.controller.cluster.datastore.DatastoreContext.Builder;
32 import org.opendaylight.controller.cluster.datastore.config.Configuration;
33 import org.opendaylight.controller.cluster.datastore.config.ConfigurationImpl;
34 import org.opendaylight.controller.cluster.datastore.config.EmptyModuleShardConfigProvider;
35 import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shard.ShardStats;
36 import org.opendaylight.controller.cluster.datastore.messages.OnDemandShardState;
37 import org.opendaylight.controller.cluster.datastore.persisted.DatastoreSnapshot;
38 import org.opendaylight.controller.cluster.datastore.utils.ActorContext;
39 import org.opendaylight.controller.cluster.raft.client.messages.GetOnDemandRaftState;
40 import org.opendaylight.controller.md.cluster.datastore.model.SchemaContextHelper;
41 import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
42 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadTransaction;
43 import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCohort;
44 import org.opendaylight.controller.sal.core.spi.data.DOMStoreTransactionChain;
45 import org.opendaylight.controller.sal.core.spi.data.DOMStoreWriteTransaction;
46 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
47 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
48 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
49 import org.slf4j.Logger;
50 import org.slf4j.LoggerFactory;
51 import scala.concurrent.Await;
52 import scala.concurrent.Future;
53 import scala.concurrent.duration.Duration;
54
55 public class IntegrationTestKit extends ShardTestKit {
56
57     private static final Logger LOG = LoggerFactory.getLogger(IntegrationTestKit.class);
58
59     protected DatastoreContext.Builder datastoreContextBuilder;
60     protected DatastoreSnapshot restoreFromSnapshot;
61     private final int commitTimeout;
62
63     public IntegrationTestKit(final ActorSystem actorSystem, final Builder datastoreContextBuilder) {
64         this(actorSystem, datastoreContextBuilder, 7);
65     }
66
67     public IntegrationTestKit(final ActorSystem actorSystem, final Builder datastoreContextBuilder, int commitTimeout) {
68         super(actorSystem);
69         this.datastoreContextBuilder = datastoreContextBuilder;
70         this.commitTimeout = commitTimeout;
71     }
72
73     public DatastoreContext.Builder getDatastoreContextBuilder() {
74         return datastoreContextBuilder;
75     }
76
77     public DistributedDataStore setupDistributedDataStore(final String typeName, final String moduleShardsConfig,
78                                                           final boolean waitUntilLeader,
79                                                           final SchemaContext schemaContext) throws Exception {
80         return setupDistributedDataStore(typeName, moduleShardsConfig, "modules.conf", waitUntilLeader, schemaContext);
81     }
82
83     public DistributedDataStore setupDistributedDataStore(final String typeName, final String moduleShardsConfig,
84                                                           final String modulesConfig,
85                                                           final boolean waitUntilLeader,
86                                                           final SchemaContext schemaContext,
87                                                           final String... shardNames) throws Exception {
88         return (DistributedDataStore) setupAbstractDataStore(DistributedDataStore.class, typeName, moduleShardsConfig,
89                 modulesConfig, waitUntilLeader, schemaContext, shardNames);
90     }
91
92     public AbstractDataStore setupAbstractDataStore(final Class<? extends AbstractDataStore> implementation,
93                                                     final String typeName, final String... shardNames)
94             throws Exception {
95         return setupAbstractDataStore(implementation, typeName, "module-shards.conf", true,
96                 SchemaContextHelper.full(), shardNames);
97     }
98
99     public AbstractDataStore setupAbstractDataStore(final Class<? extends AbstractDataStore> implementation,
100                                                     final String typeName, final boolean waitUntilLeader,
101                                                     final String... shardNames) throws Exception {
102         return setupAbstractDataStore(implementation, typeName, "module-shards.conf", waitUntilLeader,
103                 SchemaContextHelper.full(), shardNames);
104     }
105
106     public AbstractDataStore setupAbstractDataStore(final Class<? extends AbstractDataStore> implementation,
107                                                     final String typeName, final String moduleShardsConfig,
108                                                     final boolean waitUntilLeader, final String... shardNames)
109             throws Exception {
110         return setupAbstractDataStore(implementation, typeName, moduleShardsConfig, waitUntilLeader,
111                 SchemaContextHelper.full(), shardNames);
112     }
113
114     public AbstractDataStore setupAbstractDataStore(final Class<? extends AbstractDataStore> implementation,
115                                                     final String typeName, final String moduleShardsConfig,
116                                                     final boolean waitUntilLeader,
117                                                     final SchemaContext schemaContext,
118                                                     final String... shardNames) throws Exception {
119         return setupAbstractDataStore(implementation, typeName, moduleShardsConfig, "modules.conf", waitUntilLeader,
120                 schemaContext, shardNames);
121     }
122
123     private AbstractDataStore setupAbstractDataStore(final Class<? extends AbstractDataStore> implementation,
124                                                      final String typeName, final String moduleShardsConfig,
125                                                      final String modulesConfig, final boolean waitUntilLeader,
126                                                      final SchemaContext schemaContext, final String... shardNames)
127             throws Exception {
128         final ClusterWrapper cluster = new ClusterWrapperImpl(getSystem());
129         final Configuration config = new ConfigurationImpl(moduleShardsConfig, modulesConfig);
130
131         setDataStoreName(typeName);
132
133         final DatastoreContext datastoreContext = datastoreContextBuilder.build();
134         final DatastoreContextFactory mockContextFactory = Mockito.mock(DatastoreContextFactory.class);
135         Mockito.doReturn(datastoreContext).when(mockContextFactory).getBaseDatastoreContext();
136         Mockito.doReturn(datastoreContext).when(mockContextFactory).getShardDatastoreContext(Mockito.anyString());
137
138         final Constructor<? extends AbstractDataStore> constructor = implementation.getDeclaredConstructor(
139                 ActorSystem.class, ClusterWrapper.class, Configuration.class,
140                 DatastoreContextFactory.class, DatastoreSnapshot.class);
141
142         final AbstractDataStore dataStore = constructor.newInstance(
143                 getSystem(), cluster, config, mockContextFactory, restoreFromSnapshot);
144
145         dataStore.onGlobalContextUpdated(schemaContext);
146
147         if (waitUntilLeader) {
148             waitUntilLeader(dataStore.getActorContext(), shardNames);
149         }
150
151         datastoreContextBuilder = DatastoreContext.newBuilderFrom(datastoreContext);
152         return dataStore;
153     }
154
155     private void setDataStoreName(String typeName) {
156         if ("config".equals(typeName)) {
157             datastoreContextBuilder.logicalStoreType(LogicalDatastoreType.CONFIGURATION);
158         } else if ("operational".equals(typeName)) {
159             datastoreContextBuilder.logicalStoreType(LogicalDatastoreType.OPERATIONAL);
160         } else {
161             datastoreContextBuilder.dataStoreName(typeName);
162         }
163     }
164
165     public DistributedDataStore setupDistributedDataStoreWithoutConfig(final String typeName,
166                                                                        final SchemaContext schemaContext) {
167         final ClusterWrapper cluster = new ClusterWrapperImpl(getSystem());
168         final ConfigurationImpl configuration = new ConfigurationImpl(new EmptyModuleShardConfigProvider());
169
170         setDataStoreName(typeName);
171
172         final DatastoreContext datastoreContext = getDatastoreContextBuilder().build();
173
174         final DatastoreContextFactory mockContextFactory = Mockito.mock(DatastoreContextFactory.class);
175         Mockito.doReturn(datastoreContext).when(mockContextFactory).getBaseDatastoreContext();
176         Mockito.doReturn(datastoreContext).when(mockContextFactory).getShardDatastoreContext(Mockito.anyString());
177
178         final DistributedDataStore dataStore = new DistributedDataStore(getSystem(), cluster,
179                 configuration, mockContextFactory, restoreFromSnapshot);
180
181         dataStore.onGlobalContextUpdated(schemaContext);
182
183         datastoreContextBuilder = DatastoreContext.newBuilderFrom(datastoreContext);
184         return dataStore;
185     }
186
187     public DistributedDataStore setupDistributedDataStoreWithoutConfig(final String typeName,
188                                                                        final SchemaContext schemaContext,
189                                                                        final LogicalDatastoreType storeType) {
190         final ClusterWrapper cluster = new ClusterWrapperImpl(getSystem());
191         final ConfigurationImpl configuration = new ConfigurationImpl(new EmptyModuleShardConfigProvider());
192
193         setDataStoreName(typeName);
194
195         final DatastoreContext datastoreContext =
196                 getDatastoreContextBuilder().logicalStoreType(storeType).build();
197
198         final DatastoreContextFactory mockContextFactory = Mockito.mock(DatastoreContextFactory.class);
199         Mockito.doReturn(datastoreContext).when(mockContextFactory).getBaseDatastoreContext();
200         Mockito.doReturn(datastoreContext).when(mockContextFactory).getShardDatastoreContext(Mockito.anyString());
201
202         final DistributedDataStore dataStore = new DistributedDataStore(getSystem(), cluster,
203                 configuration, mockContextFactory, restoreFromSnapshot);
204
205         dataStore.onGlobalContextUpdated(schemaContext);
206
207         datastoreContextBuilder = DatastoreContext.newBuilderFrom(datastoreContext);
208         return dataStore;
209     }
210
211     public void waitUntilLeader(final ActorContext actorContext, final String... shardNames) {
212         for (String shardName: shardNames) {
213             ActorRef shard = findLocalShard(actorContext, shardName);
214
215             assertNotNull("Shard was not created for " + shardName, shard);
216
217             waitUntilLeader(shard);
218         }
219     }
220
221     public void waitUntilNoLeader(final ActorContext actorContext, final String... shardNames) {
222         for (String shardName: shardNames) {
223             ActorRef shard = findLocalShard(actorContext, shardName);
224             assertNotNull("No local shard found for " + shardName, shard);
225
226             waitUntilNoLeader(shard);
227         }
228     }
229
230     public void waitForMembersUp(final String... otherMembers) {
231         Set<String> otherMembersSet = Sets.newHashSet(otherMembers);
232         Stopwatch sw = Stopwatch.createStarted();
233         while (sw.elapsed(TimeUnit.SECONDS) <= 10) {
234             CurrentClusterState state = Cluster.get(getSystem()).state();
235             for (Member m: state.getMembers()) {
236                 if (m.status() == MemberStatus.up() && otherMembersSet.remove(m.getRoles().iterator().next())
237                         && otherMembersSet.isEmpty()) {
238                     return;
239                 }
240             }
241
242             Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
243         }
244
245         fail("Member(s) " + otherMembersSet + " are not Up");
246     }
247
248     public static ActorRef findLocalShard(final ActorContext actorContext, final String shardName) {
249         ActorRef shard = null;
250         for (int i = 0; i < 20 * 5 && shard == null; i++) {
251             Uninterruptibles.sleepUninterruptibly(50, TimeUnit.MILLISECONDS);
252             Optional<ActorRef> shardReply = actorContext.findLocalShard(shardName);
253             if (shardReply.isPresent()) {
254                 shard = shardReply.get();
255             }
256         }
257         return shard;
258     }
259
260     public static void waitUntilShardIsDown(final ActorContext actorContext, final String shardName) {
261         for (int i = 0; i < 20 * 5 ; i++) {
262             LOG.debug("Waiting for shard down {}", shardName);
263             Uninterruptibles.sleepUninterruptibly(50, TimeUnit.MILLISECONDS);
264             Optional<ActorRef> shardReply = actorContext.findLocalShard(shardName);
265             if (!shardReply.isPresent()) {
266                 return;
267             }
268         }
269
270         throw new IllegalStateException("Shard[" + shardName + " did not shutdown in time");
271     }
272
273     public static void verifyShardStats(final AbstractDataStore datastore, final String shardName,
274             final ShardStatsVerifier verifier) throws Exception {
275         ActorContext actorContext = datastore.getActorContext();
276
277         Future<ActorRef> future = actorContext.findLocalShardAsync(shardName);
278         ActorRef shardActor = Await.result(future, Duration.create(10, TimeUnit.SECONDS));
279
280         AssertionError lastError = null;
281         Stopwatch sw = Stopwatch.createStarted();
282         while (sw.elapsed(TimeUnit.SECONDS) <= 5) {
283             ShardStats shardStats = (ShardStats)actorContext
284                     .executeOperation(shardActor, Shard.GET_SHARD_MBEAN_MESSAGE);
285
286             try {
287                 verifier.verify(shardStats);
288                 return;
289             } catch (AssertionError e) {
290                 lastError = e;
291                 Uninterruptibles.sleepUninterruptibly(50, TimeUnit.MILLISECONDS);
292             }
293         }
294
295         throw lastError;
296     }
297
298     public static void verifyShardState(final AbstractDataStore datastore, final String shardName,
299             final Consumer<OnDemandShardState> verifier) throws Exception {
300         ActorContext actorContext = datastore.getActorContext();
301
302         Future<ActorRef> future = actorContext.findLocalShardAsync(shardName);
303         ActorRef shardActor = Await.result(future, Duration.create(10, TimeUnit.SECONDS));
304
305         AssertionError lastError = null;
306         Stopwatch sw = Stopwatch.createStarted();
307         while (sw.elapsed(TimeUnit.SECONDS) <= 5) {
308             OnDemandShardState shardState = (OnDemandShardState)actorContext
309                     .executeOperation(shardActor, GetOnDemandRaftState.INSTANCE);
310
311             try {
312                 verifier.accept(shardState);
313                 return;
314             } catch (AssertionError e) {
315                 lastError = e;
316                 Uninterruptibles.sleepUninterruptibly(50, TimeUnit.MILLISECONDS);
317             }
318         }
319
320         throw lastError;
321     }
322
323     void testWriteTransaction(final AbstractDataStore dataStore, final YangInstanceIdentifier nodePath,
324             final NormalizedNode<?, ?> nodeToWrite) throws Exception {
325
326         // 1. Create a write-only Tx
327
328         DOMStoreWriteTransaction writeTx = dataStore.newWriteOnlyTransaction();
329         assertNotNull("newWriteOnlyTransaction returned null", writeTx);
330
331         // 2. Write some data
332
333         writeTx.write(nodePath, nodeToWrite);
334
335         // 3. Ready the Tx for commit
336
337         DOMStoreThreePhaseCommitCohort cohort = writeTx.ready();
338
339         // 4. Commit the Tx
340
341         doCommit(cohort);
342
343         // 5. Verify the data in the store
344
345         DOMStoreReadTransaction readTx = dataStore.newReadOnlyTransaction();
346
347         Optional<NormalizedNode<?, ?>> optional = readTx.read(nodePath).get(5, TimeUnit.SECONDS);
348         assertEquals("isPresent", true, optional.isPresent());
349         assertEquals("Data node", nodeToWrite, optional.get());
350     }
351
352     public void doCommit(final DOMStoreThreePhaseCommitCohort cohort) throws Exception {
353         Boolean canCommit = cohort.canCommit().get(commitTimeout, TimeUnit.SECONDS);
354         assertEquals("canCommit", true, canCommit);
355         cohort.preCommit().get(5, TimeUnit.SECONDS);
356         cohort.commit().get(5, TimeUnit.SECONDS);
357     }
358
359     void doCommit(final ListenableFuture<Boolean> canCommitFuture, final DOMStoreThreePhaseCommitCohort cohort)
360             throws Exception {
361         Boolean canCommit = canCommitFuture.get(commitTimeout, TimeUnit.SECONDS);
362         assertEquals("canCommit", true, canCommit);
363         cohort.preCommit().get(5, TimeUnit.SECONDS);
364         cohort.commit().get(5, TimeUnit.SECONDS);
365     }
366
367     @SuppressWarnings("checkstyle:IllegalCatch")
368     void assertExceptionOnCall(final Callable<Void> callable, final Class<? extends Exception> expType)
369             throws Exception {
370         try {
371             callable.call();
372             fail("Expected " + expType.getSimpleName());
373         } catch (Exception e) {
374             assertEquals("Exception type", expType, e.getClass());
375         }
376     }
377
378     void assertExceptionOnTxChainCreates(final DOMStoreTransactionChain txChain,
379             final Class<? extends Exception> expType) throws Exception {
380         assertExceptionOnCall(() -> {
381             txChain.newWriteOnlyTransaction();
382             return null;
383         }, expType);
384
385         assertExceptionOnCall(() -> {
386             txChain.newReadWriteTransaction();
387             return null;
388         }, expType);
389
390         assertExceptionOnCall(() -> {
391             txChain.newReadOnlyTransaction();
392             return null;
393         }, expType);
394     }
395
396     public interface ShardStatsVerifier {
397         void verify(ShardStats stats);
398     }
399 }