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