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