b86c7d145c9d732d9401b32dffe7aa68ef4dfadb
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / test / java / org / opendaylight / controller / cluster / datastore / compat / PreLithiumShardTest.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.compat;
9
10 import static org.junit.Assert.assertEquals;
11 import static org.junit.Assert.assertNotNull;
12 import static org.junit.Assert.assertTrue;
13 import static org.mockito.Mockito.inOrder;
14 import static org.opendaylight.controller.cluster.datastore.DataStoreVersions.HELIUM_2_VERSION;
15 import akka.actor.ActorRef;
16 import akka.actor.PoisonPill;
17 import akka.dispatch.Dispatchers;
18 import akka.dispatch.OnComplete;
19 import akka.pattern.Patterns;
20 import akka.testkit.TestActorRef;
21 import akka.util.Timeout;
22 import com.google.common.base.Optional;
23 import java.io.IOException;
24 import java.util.Collections;
25 import java.util.HashSet;
26 import java.util.Set;
27 import java.util.concurrent.CountDownLatch;
28 import java.util.concurrent.TimeUnit;
29 import java.util.concurrent.atomic.AtomicReference;
30 import org.junit.Test;
31 import org.mockito.InOrder;
32 import org.opendaylight.controller.cluster.datastore.AbstractShardTest;
33 import org.opendaylight.controller.cluster.datastore.Shard;
34 import org.opendaylight.controller.cluster.datastore.ShardDataTree;
35 import org.opendaylight.controller.cluster.datastore.ShardDataTreeCohort;
36 import org.opendaylight.controller.cluster.datastore.ShardTestKit;
37 import org.opendaylight.controller.cluster.datastore.messages.CanCommitTransaction;
38 import org.opendaylight.controller.cluster.datastore.messages.CanCommitTransactionReply;
39 import org.opendaylight.controller.cluster.datastore.messages.CommitTransaction;
40 import org.opendaylight.controller.cluster.datastore.messages.CommitTransactionReply;
41 import org.opendaylight.controller.cluster.datastore.messages.ForwardedReadyTransaction;
42 import org.opendaylight.controller.cluster.datastore.messages.ReadyTransactionReply;
43 import org.opendaylight.controller.cluster.datastore.modification.MergeModification;
44 import org.opendaylight.controller.cluster.datastore.modification.Modification;
45 import org.opendaylight.controller.cluster.datastore.modification.ModificationPayload;
46 import org.opendaylight.controller.cluster.datastore.modification.MutableCompositeModification;
47 import org.opendaylight.controller.cluster.datastore.modification.WriteModification;
48 import org.opendaylight.controller.cluster.datastore.node.NormalizedNodeToNodeCodec;
49 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
50 import org.opendaylight.controller.cluster.raft.ReplicatedLogImplEntry;
51 import org.opendaylight.controller.cluster.raft.Snapshot;
52 import org.opendaylight.controller.cluster.raft.base.messages.ApplyLogEntries;
53 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
54 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.CompositeModificationByteStringPayload;
55 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.CompositeModificationPayload;
56 import org.opendaylight.controller.cluster.raft.utils.InMemoryJournal;
57 import org.opendaylight.controller.cluster.raft.utils.InMemorySnapshotStore;
58 import org.opendaylight.controller.md.cluster.datastore.model.TestModel;
59 import org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages;
60 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
61 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier.PathArgument;
62 import org.opendaylight.yangtools.yang.data.api.schema.DataContainerChild;
63 import org.opendaylight.yangtools.yang.data.api.schema.MapEntryNode;
64 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
65 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTree;
66 import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes;
67 import org.opendaylight.yangtools.yang.data.impl.schema.tree.InMemoryDataTreeFactory;
68 import scala.concurrent.Future;
69 import scala.concurrent.duration.FiniteDuration;
70
71 /**
72  * Unit tests for backwards compatibility with pre-Lithium versions.
73  *
74  * @author Thomas Pantelis
75  */
76 public class PreLithiumShardTest extends AbstractShardTest {
77
78     private static CompositeModificationPayload newLegacyPayload(final Modification... mods) {
79         MutableCompositeModification compMod = new MutableCompositeModification();
80         for(Modification mod: mods) {
81             compMod.addModification(mod);
82         }
83
84         return new CompositeModificationPayload(compMod.toSerializable());
85     }
86
87     private static CompositeModificationByteStringPayload newLegacyByteStringPayload(final Modification... mods) {
88         MutableCompositeModification compMod = new MutableCompositeModification();
89         for(Modification mod: mods) {
90             compMod.addModification(mod);
91         }
92
93         return new CompositeModificationByteStringPayload(compMod.toSerializable());
94     }
95
96     private static ModificationPayload newModificationPayload(final Modification... mods) throws IOException {
97         MutableCompositeModification compMod = new MutableCompositeModification();
98         for(Modification mod: mods) {
99             compMod.addModification(mod);
100         }
101
102         return new ModificationPayload(compMod);
103     }
104
105     @Test
106     public void testApplyHelium2VersionSnapshot() throws Exception {
107         TestActorRef<Shard> shard = TestActorRef.create(getSystem(), newShardProps(),
108                 "testApplyHelium2VersionSnapshot");
109
110         NormalizedNodeToNodeCodec codec = new NormalizedNodeToNodeCodec(SCHEMA_CONTEXT);
111
112         DataTree store = InMemoryDataTreeFactory.getInstance().create();
113         store.setSchemaContext(SCHEMA_CONTEXT);
114
115         writeToStore(store, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
116
117         YangInstanceIdentifier root = YangInstanceIdentifier.builder().build();
118         NormalizedNode<?,?> expected = readStore(store, root);
119
120         NormalizedNodeMessages.Container encode = codec.encode(expected);
121
122         Snapshot snapshot = Snapshot.create(encode.getNormalizedNode().toByteString().toByteArray(),
123                 Collections.<ReplicatedLogEntry>emptyList(), 1, 2, 3, 4);
124
125         shard.underlyingActor().getRaftActorSnapshotCohort().applySnapshot(snapshot.getState());
126
127         NormalizedNode<?,?> actual = readStore(shard, root);
128
129         assertEquals("Root node", expected, actual);
130
131         shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
132     }
133
134     @Test
135     public void testHelium2VersionApplyStateLegacy() throws Exception {
136         new ShardTestKit(getSystem()) {{
137             TestActorRef<Shard> shard = TestActorRef.create(getSystem(), newShardProps(),
138                     "testHelium2VersionApplyStateLegacy");
139
140             waitUntilLeader(shard);
141
142             NormalizedNode<?, ?> node = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
143
144             ApplyState applyState = new ApplyState(null, "test", new ReplicatedLogImplEntry(1, 2,
145                     newLegacyByteStringPayload(new WriteModification(TestModel.TEST_PATH, node))));
146
147             shard.underlyingActor().onReceiveCommand(applyState);
148
149             NormalizedNode<?,?> actual = readStore(shard, TestModel.TEST_PATH);
150             assertEquals("Applied state", node, actual);
151
152             shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
153         }};
154     }
155
156     @Test
157     public void testHelium2VersionRecovery() throws Exception {
158
159         DataTree testStore = InMemoryDataTreeFactory.getInstance().create();
160         testStore.setSchemaContext(SCHEMA_CONTEXT);
161
162         writeToStore(testStore, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
163
164         NormalizedNode<?, ?> root = readStore(testStore, YangInstanceIdentifier.builder().build());
165
166         InMemorySnapshotStore.addSnapshot(shardID.toString(), Snapshot.create(
167                 new NormalizedNodeToNodeCodec(SCHEMA_CONTEXT).encode(root).
168                                 getNormalizedNode().toByteString().toByteArray(),
169                                 Collections.<ReplicatedLogEntry>emptyList(), 0, 1, -1, -1));
170
171         InMemoryJournal.addEntry(shardID.toString(), 0, new String("Dummy data as snapshot sequence number is " +
172                 "set to 0 in InMemorySnapshotStore and journal recovery seq number will start from 1"));
173
174         // Set up the InMemoryJournal.
175
176         InMemoryJournal.addEntry(shardID.toString(), 1, new ReplicatedLogImplEntry(0, 1, newLegacyPayload(
177                   new WriteModification(TestModel.OUTER_LIST_PATH,
178                           ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build()))));
179
180         int nListEntries = 16;
181         Set<Integer> listEntryKeys = new HashSet<>();
182         int i = 1;
183
184         // Add some CompositeModificationPayload entries
185         for(; i <= 8; i++) {
186             listEntryKeys.add(Integer.valueOf(i));
187             YangInstanceIdentifier path = YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH)
188                     .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, i).build();
189             Modification mod = new MergeModification(path,
190                     ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, i));
191             InMemoryJournal.addEntry(shardID.toString(), i+1, new ReplicatedLogImplEntry(i, 1,
192                     newLegacyPayload(mod)));
193         }
194
195         // Add some CompositeModificationByteStringPayload entries
196         for(; i <= nListEntries; i++) {
197             listEntryKeys.add(Integer.valueOf(i));
198             YangInstanceIdentifier path = YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH)
199                     .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, i).build();
200             Modification mod = new MergeModification(path,
201                     ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, i));
202             InMemoryJournal.addEntry(shardID.toString(), i+1, new ReplicatedLogImplEntry(i, 1,
203                     newLegacyByteStringPayload(mod)));
204         }
205
206         InMemoryJournal.addEntry(shardID.toString(), nListEntries + 2, new ApplyLogEntries(nListEntries));
207
208         testRecovery(listEntryKeys);
209     }
210
211     @SuppressWarnings({ "unchecked" })
212     @Test
213     public void testPreLithiumConcurrentThreePhaseCommits() throws Throwable {
214         new ShardTestKit(getSystem()) {{
215             final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
216                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
217                     "testPreLithiumConcurrentThreePhaseCommits");
218
219             waitUntilLeader(shard);
220
221             // Setup 3 simulated transactions with mock cohorts backed by real cohorts.
222
223             ShardDataTree dataStore = shard.underlyingActor().getDataStore();
224
225             String transactionID1 = "tx1";
226             MutableCompositeModification modification1 = new MutableCompositeModification();
227             ShardDataTreeCohort cohort1 = setupMockWriteTransaction("cohort1", dataStore,
228                     TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), modification1);
229
230             String transactionID2 = "tx2";
231             MutableCompositeModification modification2 = new MutableCompositeModification();
232             ShardDataTreeCohort cohort2 = setupMockWriteTransaction("cohort2", dataStore,
233                     TestModel.OUTER_LIST_PATH,
234                     ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(),
235                     modification2);
236
237             String transactionID3 = "tx3";
238             MutableCompositeModification modification3 = new MutableCompositeModification();
239             ShardDataTreeCohort cohort3 = setupMockWriteTransaction("cohort3", dataStore,
240                     YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH)
241                         .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1).build(),
242                     ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1),
243                     modification3);
244
245             long timeoutSec = 5;
246             final FiniteDuration duration = FiniteDuration.create(timeoutSec, TimeUnit.SECONDS);
247             final Timeout timeout = new Timeout(duration);
248
249             // Simulate the ForwardedReadyTransaction message for the first Tx that would be sent
250             // by the ShardTransaction.
251
252             shard.tell(new ForwardedReadyTransaction(transactionID1, HELIUM_2_VERSION,
253                     cohort1, modification1, true, false), getRef());
254             ReadyTransactionReply readyReply = ReadyTransactionReply.fromSerializable(
255                     expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS));
256             assertEquals("Cohort path", shard.path().toString(), readyReply.getCohortPath());
257
258             // Send the CanCommitTransaction message for the first Tx.
259
260             shard.tell(new CanCommitTransaction(transactionID1).toSerializable(), getRef());
261             CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
262                     expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS));
263             assertEquals("Can commit", true, canCommitReply.getCanCommit());
264
265             // Send the ForwardedReadyTransaction for the next 2 Tx's.
266
267             shard.tell(new ForwardedReadyTransaction(transactionID2, HELIUM_2_VERSION,
268                     cohort2, modification2, true, false), getRef());
269             expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS);
270
271             shard.tell(new ForwardedReadyTransaction(transactionID3, HELIUM_2_VERSION,
272                     cohort3, modification3, true, false), getRef());
273             expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS);
274
275             // Send the CanCommitTransaction message for the next 2 Tx's. These should get queued and
276             // processed after the first Tx completes.
277
278             Future<Object> canCommitFuture1 = Patterns.ask(shard,
279                     new CanCommitTransaction(transactionID2).toSerializable(), timeout);
280
281             Future<Object> canCommitFuture2 = Patterns.ask(shard,
282                     new CanCommitTransaction(transactionID3).toSerializable(), timeout);
283
284             // Send the CommitTransaction message for the first Tx. After it completes, it should
285             // trigger the 2nd Tx to proceed which should in turn then trigger the 3rd.
286
287             shard.tell(new CommitTransaction(transactionID1).toSerializable(), getRef());
288             expectMsgClass(duration, CommitTransactionReply.SERIALIZABLE_CLASS);
289
290             // Wait for the next 2 Tx's to complete.
291
292             final AtomicReference<Throwable> caughtEx = new AtomicReference<>();
293             final CountDownLatch commitLatch = new CountDownLatch(2);
294
295             class OnFutureComplete extends OnComplete<Object> {
296                 private final Class<?> expRespType;
297
298                 OnFutureComplete(final Class<?> expRespType) {
299                     this.expRespType = expRespType;
300                 }
301
302                 @Override
303                 public void onComplete(final Throwable error, final Object resp) {
304                     if(error != null) {
305                         caughtEx.set(new AssertionError(getClass().getSimpleName() + " failure", error));
306                     } else {
307                         try {
308                             assertEquals("Commit response type", expRespType, resp.getClass());
309                             onSuccess(resp);
310                         } catch (Exception e) {
311                             caughtEx.set(e);
312                         }
313                     }
314                 }
315
316                 void onSuccess(final Object resp) throws Exception {
317                 }
318             }
319
320             class OnCommitFutureComplete extends OnFutureComplete {
321                 OnCommitFutureComplete() {
322                     super(CommitTransactionReply.SERIALIZABLE_CLASS);
323                 }
324
325                 @Override
326                 public void onComplete(final Throwable error, final Object resp) {
327                     super.onComplete(error, resp);
328                     commitLatch.countDown();
329                 }
330             }
331
332             class OnCanCommitFutureComplete extends OnFutureComplete {
333                 private final String transactionID;
334
335                 OnCanCommitFutureComplete(final String transactionID) {
336                     super(CanCommitTransactionReply.SERIALIZABLE_CLASS);
337                     this.transactionID = transactionID;
338                 }
339
340                 @Override
341                 void onSuccess(final Object resp) throws Exception {
342                     CanCommitTransactionReply canCommitReply =
343                             CanCommitTransactionReply.fromSerializable(resp);
344                     assertEquals("Can commit", true, canCommitReply.getCanCommit());
345
346                     Future<Object> commitFuture = Patterns.ask(shard,
347                             new CommitTransaction(transactionID).toSerializable(), timeout);
348                     commitFuture.onComplete(new OnCommitFutureComplete(), getSystem().dispatcher());
349                 }
350             }
351
352             canCommitFuture1.onComplete(new OnCanCommitFutureComplete(transactionID2),
353                     getSystem().dispatcher());
354
355             canCommitFuture2.onComplete(new OnCanCommitFutureComplete(transactionID3),
356                     getSystem().dispatcher());
357
358             boolean done = commitLatch.await(timeoutSec, TimeUnit.SECONDS);
359
360             if(caughtEx.get() != null) {
361                 throw caughtEx.get();
362             }
363
364             assertEquals("Commits complete", true, done);
365
366             InOrder inOrder = inOrder(cohort1, cohort2, cohort3);
367             inOrder.verify(cohort1).canCommit();
368             inOrder.verify(cohort1).preCommit();
369             inOrder.verify(cohort1).commit();
370             inOrder.verify(cohort2).canCommit();
371             inOrder.verify(cohort2).preCommit();
372             inOrder.verify(cohort2).commit();
373             inOrder.verify(cohort3).canCommit();
374             inOrder.verify(cohort3).preCommit();
375             inOrder.verify(cohort3).commit();
376
377             // Verify data in the data store.
378
379             NormalizedNode<?, ?> outerList = readStore(shard, TestModel.OUTER_LIST_PATH);
380             assertNotNull(TestModel.OUTER_LIST_QNAME.getLocalName() + " not found", outerList);
381             assertTrue(TestModel.OUTER_LIST_QNAME.getLocalName() + " value is not Iterable",
382                     outerList.getValue() instanceof Iterable);
383             Object entry = ((Iterable<Object>)outerList.getValue()).iterator().next();
384             assertTrue(TestModel.OUTER_LIST_QNAME.getLocalName() + " entry is not MapEntryNode",
385                        entry instanceof MapEntryNode);
386             MapEntryNode mapEntry = (MapEntryNode)entry;
387             Optional<DataContainerChild<? extends PathArgument, ?>> idLeaf =
388                     mapEntry.getChild(new YangInstanceIdentifier.NodeIdentifier(TestModel.ID_QNAME));
389             assertTrue("Missing leaf " + TestModel.ID_QNAME.getLocalName(), idLeaf.isPresent());
390             assertEquals(TestModel.ID_QNAME.getLocalName() + " value", 1, idLeaf.get().getValue());
391
392             verifyLastApplied(shard, 2);
393
394             shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
395         }};
396     }
397 }