Snapshot and journal export on recovery
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / test / java / org / opendaylight / controller / cluster / datastore / AbstractShardTest.java
index ab7e8f040159471c3290713797d2105a321584a8..31f2b6b0674a6be403468f364487e861f111c00b 100644 (file)
@@ -11,7 +11,8 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.nullable;
 import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
@@ -28,7 +29,6 @@ import akka.japi.Creator;
 import akka.pattern.Patterns;
 import akka.testkit.TestActorRef;
 import akka.util.Timeout;
-import com.google.common.base.Optional;
 import com.google.common.primitives.UnsignedLong;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Uninterruptibles;
@@ -36,7 +36,9 @@ import java.io.IOException;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
+import java.util.SortedSet;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
@@ -55,16 +57,16 @@ import org.opendaylight.controller.cluster.datastore.modification.MergeModificat
 import org.opendaylight.controller.cluster.datastore.modification.MutableCompositeModification;
 import org.opendaylight.controller.cluster.datastore.modification.WriteModification;
 import org.opendaylight.controller.cluster.datastore.persisted.CommitTransactionPayload;
-import org.opendaylight.controller.cluster.datastore.persisted.PreBoronShardDataTreeSnapshot;
+import org.opendaylight.controller.cluster.datastore.persisted.MetadataShardDataTreeSnapshot;
+import org.opendaylight.controller.cluster.datastore.persisted.ShardSnapshotState;
 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
-import org.opendaylight.controller.cluster.raft.Snapshot;
 import org.opendaylight.controller.cluster.raft.TestActorFactory;
+import org.opendaylight.controller.cluster.raft.persisted.Snapshot;
 import org.opendaylight.controller.cluster.raft.utils.InMemoryJournal;
 import org.opendaylight.controller.cluster.raft.utils.InMemorySnapshotStore;
 import org.opendaylight.controller.md.cluster.datastore.model.CarsModel;
 import org.opendaylight.controller.md.cluster.datastore.model.TestModel;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
-import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier.PathArgument;
 import org.opendaylight.yangtools.yang.data.api.schema.DataContainerChild;
 import org.opendaylight.yangtools.yang.data.api.schema.MapEntryNode;
 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
@@ -72,17 +74,16 @@ import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTree;
 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidate;
 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidateNode;
 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidateTip;
+import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeConfiguration;
 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeModification;
 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataValidationFailedException;
 import org.opendaylight.yangtools.yang.data.api.schema.tree.ModificationType;
-import org.opendaylight.yangtools.yang.data.api.schema.tree.TipProducingDataTree;
-import org.opendaylight.yangtools.yang.data.api.schema.tree.TreeType;
 import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes;
 import org.opendaylight.yangtools.yang.data.impl.schema.tree.InMemoryDataTreeFactory;
-import org.opendaylight.yangtools.yang.model.api.SchemaContext;
+import org.opendaylight.yangtools.yang.model.api.EffectiveModelContext;
 import scala.concurrent.Await;
 import scala.concurrent.Future;
-import scala.concurrent.duration.Duration;
+import scala.concurrent.duration.FiniteDuration;
 
 /**
  * Abstract base for shard unit tests.
@@ -90,20 +91,22 @@ import scala.concurrent.duration.Duration;
  * @author Thomas Pantelis
  */
 public abstract class AbstractShardTest extends AbstractActorTest {
-    protected static final SchemaContext SCHEMA_CONTEXT = TestModel.createTestContext();
+    protected static final EffectiveModelContext SCHEMA_CONTEXT = TestModel.createTestContext();
 
-    private static final AtomicInteger NEXT_SHARD_NUM = new AtomicInteger();
-
-    protected final ShardIdentifier shardID = ShardIdentifier.create("inventory", MemberName.forName("member-1"),
-        "config" + NEXT_SHARD_NUM.getAndIncrement());
+    protected static final AtomicInteger SHARD_NUM = new AtomicInteger();
+    protected static final int HEARTBEAT_MILLIS = 100;
 
     protected final Builder dataStoreContextBuilder = DatastoreContext.newBuilder()
-            .shardJournalRecoveryLogBatchSize(3).shardSnapshotBatchCount(5000).shardHeartbeatIntervalInMillis(100);
+            .shardJournalRecoveryLogBatchSize(3).shardSnapshotBatchCount(5000)
+            .shardHeartbeatIntervalInMillis(HEARTBEAT_MILLIS);
 
     protected final TestActorFactory actorFactory = new TestActorFactory(getSystem());
+    protected final int nextShardNum = SHARD_NUM.getAndIncrement();
+    protected final ShardIdentifier shardID = ShardIdentifier.create("inventory", MemberName.forName("member-1"),
+        "config" + nextShardNum);
 
     @Before
-    public void setUp() {
+    public void setUp() throws Exception {
         InMemorySnapshotStore.clear();
         InMemoryJournal.clear();
     }
@@ -124,10 +127,11 @@ public abstract class AbstractShardTest extends AbstractActorTest {
     }
 
     protected Shard.Builder newShardBuilder() {
-        return Shard.builder().id(shardID).datastoreContext(newDatastoreContext()).schemaContext(SCHEMA_CONTEXT);
+        return Shard.builder().id(shardID).datastoreContext(newDatastoreContext())
+            .schemaContextProvider(() -> SCHEMA_CONTEXT);
     }
 
-    protected void testRecovery(final Set<Integer> listEntryKeys) throws Exception {
+    protected void testRecovery(final Set<Integer> listEntryKeys, final boolean stopActorOnFinish) throws Exception {
         // Create the actor and wait for recovery complete.
 
         final int nListEntries = listEntryKeys.size();
@@ -145,26 +149,25 @@ public abstract class AbstractShardTest extends AbstractActorTest {
             }
         };
 
-        final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
-                Props.create(new DelegatingShardCreator(creator)).withDispatcher(Dispatchers.DefaultDispatcherId()),
-                "testRecovery");
+        final TestActorRef<Shard> shard = TestActorRef.create(getSystem(), Props.create(Shard.class,
+                new DelegatingShardCreator(creator)).withDispatcher(Dispatchers.DefaultDispatcherId()), "testRecovery");
 
-        assertEquals("Recovery complete", true, recoveryComplete.await(5, TimeUnit.SECONDS));
+        assertTrue("Recovery complete", recoveryComplete.await(5, TimeUnit.SECONDS));
 
         // Verify data in the data store.
 
-        final NormalizedNode<?, ?> outerList = readStore(shard, TestModel.OUTER_LIST_PATH);
+        final NormalizedNode outerList = readStore(shard, TestModel.OUTER_LIST_PATH);
         assertNotNull(TestModel.OUTER_LIST_QNAME.getLocalName() + " not found", outerList);
         assertTrue(TestModel.OUTER_LIST_QNAME.getLocalName() + " value is not Iterable",
-                outerList.getValue() instanceof Iterable);
-        for (final Object entry: (Iterable<?>) outerList.getValue()) {
+                outerList.body() instanceof Iterable);
+        for (final Object entry: (Iterable<?>) outerList.body()) {
             assertTrue(TestModel.OUTER_LIST_QNAME.getLocalName() + " entry is not MapEntryNode",
                     entry instanceof MapEntryNode);
             final MapEntryNode mapEntry = (MapEntryNode)entry;
-            final Optional<DataContainerChild<? extends PathArgument, ?>> idLeaf =
-                    mapEntry.getChild(new YangInstanceIdentifier.NodeIdentifier(TestModel.ID_QNAME));
+            final Optional<DataContainerChild> idLeaf =
+                    mapEntry.findChildByArg(new YangInstanceIdentifier.NodeIdentifier(TestModel.ID_QNAME));
             assertTrue("Missing leaf " + TestModel.ID_QNAME.getLocalName(), idLeaf.isPresent());
-            final Object value = idLeaf.get().getValue();
+            final Object value = idLeaf.get().body();
             assertTrue("Unexpected value for leaf " + TestModel.ID_QNAME.getLocalName() + ": " + value,
                     listEntryKeys.remove(value));
         }
@@ -180,7 +183,9 @@ public abstract class AbstractShardTest extends AbstractActorTest {
         assertEquals("Last applied", nListEntries,
                 shard.underlyingActor().getShardMBean().getLastApplied());
 
-        shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
+        if (stopActorOnFinish) {
+            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
+        }
     }
 
     protected void verifyLastApplied(final TestActorRef<Shard> shard, final long expectedValue) {
@@ -196,36 +201,31 @@ public abstract class AbstractShardTest extends AbstractActorTest {
         Assert.fail(String.format("Expected last applied: %d, Actual: %d", expectedValue, lastApplied));
     }
 
-    protected TipProducingDataTree createDelegatingMockDataTree() throws Exception {
-        TipProducingDataTree actual = InMemoryDataTreeFactory.getInstance().create(TreeType.CONFIGURATION);
-        final TipProducingDataTree mock = mock(TipProducingDataTree.class);
+    protected DataTree createDelegatingMockDataTree() throws Exception {
+        final DataTree actual = new InMemoryDataTreeFactory().create(DataTreeConfiguration.DEFAULT_CONFIGURATION);
+        final DataTree mock = mock(DataTree.class);
 
         doAnswer(invocation -> {
-            actual.validate(invocation.getArgumentAt(0, DataTreeModification.class));
+            actual.validate(invocation.getArgument(0));
             return null;
         }).when(mock).validate(any(DataTreeModification.class));
 
-        doAnswer(invocation -> {
-            return actual.prepare(invocation.getArgumentAt(0, DataTreeModification.class));
-        }).when(mock).prepare(any(DataTreeModification.class));
+        doAnswer(invocation -> actual.prepare(invocation.getArgument(0))).when(
+                mock).prepare(any(DataTreeModification.class));
 
         doAnswer(invocation -> {
-            actual.commit(invocation.getArgumentAt(0, DataTreeCandidate.class));
+            actual.commit(invocation.getArgument(0));
             return null;
         }).when(mock).commit(any(DataTreeCandidate.class));
 
         doAnswer(invocation -> {
-            actual.setSchemaContext(invocation.getArgumentAt(0, SchemaContext.class));
+            actual.setEffectiveModelContext(invocation.getArgument(0));
             return null;
-        }).when(mock).setSchemaContext(any(SchemaContext.class));
+        }).when(mock).setEffectiveModelContext(any(EffectiveModelContext.class));
 
-        doAnswer(invocation -> {
-            return actual.takeSnapshot();
-        }).when(mock).takeSnapshot();
+        doAnswer(invocation -> actual.takeSnapshot()).when(mock).takeSnapshot();
 
-        doAnswer(invocation -> {
-            return actual.getRootPath();
-        }).when(mock).getRootPath();
+        doAnswer(invocation -> actual.getRootPath()).when(mock).getRootPath();
 
         return mock;
     }
@@ -266,14 +266,14 @@ public abstract class AbstractShardTest extends AbstractActorTest {
                                                              final boolean doCommitOnReady) {
         final BatchedModifications batchedModifications = new BatchedModifications(transactionID, CURRENT_VERSION);
         batchedModifications.addModification(modification);
-        batchedModifications.setReady(true);
+        batchedModifications.setReady();
         batchedModifications.setDoCommitOnReady(doCommitOnReady);
         batchedModifications.setTotalMessagesSent(1);
         return batchedModifications;
     }
 
     protected static BatchedModifications prepareBatchedModifications(final TransactionIdentifier transactionID,
-            final YangInstanceIdentifier path, final NormalizedNode<?, ?> data, final boolean doCommitOnReady) {
+            final YangInstanceIdentifier path, final NormalizedNode data, final boolean doCommitOnReady) {
         final MutableCompositeModification modification = new MutableCompositeModification();
         modification.addModification(new WriteModification(path, data));
         return prepareBatchedModifications(transactionID, modification, doCommitOnReady);
@@ -281,43 +281,43 @@ public abstract class AbstractShardTest extends AbstractActorTest {
 
     protected static ForwardedReadyTransaction prepareForwardedReadyTransaction(final TestActorRef<Shard> shard,
             final TransactionIdentifier transactionID, final YangInstanceIdentifier path,
-            final NormalizedNode<?, ?> data, final boolean doCommitOnReady) {
+            final NormalizedNode data, final boolean doCommitOnReady) {
         ReadWriteShardDataTreeTransaction rwTx = shard.underlyingActor().getDataStore()
                 .newReadWriteTransaction(transactionID);
         rwTx.getSnapshot().write(path, data);
-        return new ForwardedReadyTransaction(transactionID, CURRENT_VERSION, rwTx, doCommitOnReady);
+        return new ForwardedReadyTransaction(transactionID, CURRENT_VERSION, rwTx, doCommitOnReady, Optional.empty());
     }
 
-    public static NormalizedNode<?,?> readStore(final TestActorRef<? extends Shard> shard,
-            final YangInstanceIdentifier id) throws ExecutionException, InterruptedException {
-        return shard.underlyingActor().getDataStore().readNode(id).orNull();
+    public static NormalizedNode readStore(final TestActorRef<? extends Shard> shard,
+            final YangInstanceIdentifier id) {
+        return shard.underlyingActor().getDataStore().readNode(id).orElse(null);
     }
 
-    public static NormalizedNode<?,?> readStore(final DataTree store, final YangInstanceIdentifier id) {
-        return store.takeSnapshot().readNode(id).orNull();
+    public static NormalizedNode readStore(final DataTree store, final YangInstanceIdentifier id) {
+        return store.takeSnapshot().readNode(id).orElse(null);
     }
 
     public void writeToStore(final TestActorRef<Shard> shard, final YangInstanceIdentifier id,
-            final NormalizedNode<?,?> node) throws InterruptedException, ExecutionException {
+            final NormalizedNode node) throws InterruptedException, ExecutionException {
         Future<Object> future = Patterns.ask(shard, newBatchedModifications(nextTransactionId(),
                 id, node, true, true, 1), new Timeout(5, TimeUnit.SECONDS));
         try {
-            Await.ready(future, Duration.create(5, TimeUnit.SECONDS));
+            Await.ready(future, FiniteDuration.create(5, TimeUnit.SECONDS));
         } catch (TimeoutException e) {
             throw new ExecutionException(e);
         }
     }
 
     public static void writeToStore(final ShardDataTree store, final YangInstanceIdentifier id,
-            final NormalizedNode<?,?> node) throws Exception {
+            final NormalizedNode node) throws DataValidationFailedException {
         BatchedModifications batched = newBatchedModifications(nextTransactionId(), id, node, true, true, 1);
         DataTreeModification modification = store.getDataTree().takeSnapshot().newModification();
         batched.apply(modification);
-        store.commit(modification);
+        store.notifyListeners(commitTransaction(store.getDataTree(), modification));
     }
 
-    public static void writeToStore(final DataTree store, final YangInstanceIdentifier id,
-            final NormalizedNode<?,?> node) throws DataValidationFailedException {
+    public static void writeToStore(final DataTree store, final YangInstanceIdentifier id, final NormalizedNode node)
+            throws DataValidationFailedException {
         final DataTreeModification transaction = store.takeSnapshot().newModification();
 
         transaction.write(id, node);
@@ -327,30 +327,30 @@ public abstract class AbstractShardTest extends AbstractActorTest {
         store.commit(candidate);
     }
 
-    public void mergeToStore(final ShardDataTree store, final YangInstanceIdentifier id,
-            final NormalizedNode<?,?> node) throws Exception {
+    public void mergeToStore(final ShardDataTree store, final YangInstanceIdentifier id, final NormalizedNode node)
+        throws DataValidationFailedException {
         final BatchedModifications batched = new BatchedModifications(nextTransactionId(), CURRENT_VERSION);
         batched.addModification(new MergeModification(id, node));
-        batched.setReady(true);
+        batched.setReady();
         batched.setDoCommitOnReady(true);
         batched.setTotalMessagesSent(1);
 
         DataTreeModification modification = store.getDataTree().takeSnapshot().newModification();
         batched.apply(modification);
-        store.commit(modification);
+        store.notifyListeners(commitTransaction(store.getDataTree(), modification));
     }
 
     DataTree setupInMemorySnapshotStore() throws DataValidationFailedException {
-        final DataTree testStore = InMemoryDataTreeFactory.getInstance().create(TreeType.OPERATIONAL);
-        testStore.setSchemaContext(SCHEMA_CONTEXT);
+        final DataTree testStore = new InMemoryDataTreeFactory().create(
+            DataTreeConfiguration.DEFAULT_OPERATIONAL, SCHEMA_CONTEXT);
 
         writeToStore(testStore, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
 
-        final NormalizedNode<?, ?> root = readStore(testStore, YangInstanceIdentifier.EMPTY);
+        final NormalizedNode root = readStore(testStore, YangInstanceIdentifier.empty());
 
         InMemorySnapshotStore.addSnapshot(shardID.toString(), Snapshot.create(
-                new PreBoronShardDataTreeSnapshot(root).serialize(),
-                Collections.<ReplicatedLogEntry>emptyList(), 0, 1, -1, -1));
+                new ShardSnapshotState(new MetadataShardDataTreeSnapshot(root)),
+                Collections.<ReplicatedLogEntry>emptyList(), 0, 1, -1, -1, 1, null, null));
         return testStore;
     }
 
@@ -363,30 +363,42 @@ public abstract class AbstractShardTest extends AbstractActorTest {
     }
 
     static BatchedModifications newBatchedModifications(final TransactionIdentifier transactionID,
-            final YangInstanceIdentifier path, final NormalizedNode<?, ?> data, final boolean ready,
+            final YangInstanceIdentifier path, final NormalizedNode data, final boolean ready,
             final boolean doCommitOnReady, final int messagesSent) {
         final BatchedModifications batched = new BatchedModifications(transactionID, CURRENT_VERSION);
         batched.addModification(new WriteModification(path, data));
-        batched.setReady(ready);
+        if (ready) {
+            batched.setReady();
+        }
         batched.setDoCommitOnReady(doCommitOnReady);
         batched.setTotalMessagesSent(messagesSent);
         return batched;
     }
 
+    static BatchedModifications newReadyBatchedModifications(final TransactionIdentifier transactionID,
+            final YangInstanceIdentifier path, final NormalizedNode data,
+            final SortedSet<String> participatingShardNames) {
+        final BatchedModifications batched = new BatchedModifications(transactionID, CURRENT_VERSION);
+        batched.addModification(new WriteModification(path, data));
+        batched.setReady(Optional.of(participatingShardNames));
+        batched.setTotalMessagesSent(1);
+        return batched;
+    }
+
     @SuppressWarnings("unchecked")
-    static void verifyOuterListEntry(final TestActorRef<Shard> shard, final Object expIDValue) throws Exception {
-        final NormalizedNode<?, ?> outerList = readStore(shard, TestModel.OUTER_LIST_PATH);
+    static void verifyOuterListEntry(final TestActorRef<Shard> shard, final Object expIDValue) {
+        final NormalizedNode outerList = readStore(shard, TestModel.OUTER_LIST_PATH);
         assertNotNull(TestModel.OUTER_LIST_QNAME.getLocalName() + " not found", outerList);
         assertTrue(TestModel.OUTER_LIST_QNAME.getLocalName() + " value is not Iterable",
-                outerList.getValue() instanceof Iterable);
-        final Object entry = ((Iterable<Object>)outerList.getValue()).iterator().next();
+                outerList.body() instanceof Iterable);
+        final Object entry = ((Iterable<Object>)outerList.body()).iterator().next();
         assertTrue(TestModel.OUTER_LIST_QNAME.getLocalName() + " entry is not MapEntryNode",
                 entry instanceof MapEntryNode);
         final MapEntryNode mapEntry = (MapEntryNode)entry;
-        final Optional<DataContainerChild<? extends PathArgument, ?>> idLeaf =
-                mapEntry.getChild(new YangInstanceIdentifier.NodeIdentifier(TestModel.ID_QNAME));
+        final Optional<DataContainerChild> idLeaf =
+                mapEntry.findChildByArg(new YangInstanceIdentifier.NodeIdentifier(TestModel.ID_QNAME));
         assertTrue("Missing leaf " + TestModel.ID_QNAME.getLocalName(), idLeaf.isPresent());
-        assertEquals(TestModel.ID_QNAME.getLocalName() + " value", expIDValue, idLeaf.get().getValue());
+        assertEquals(TestModel.ID_QNAME.getLocalName() + " value", expIDValue, idLeaf.get().body());
     }
 
     public static DataTreeCandidateTip mockCandidate(final String name) {
@@ -404,16 +416,18 @@ public abstract class AbstractShardTest extends AbstractActorTest {
         final DataTreeCandidateTip mockCandidate = mock(DataTreeCandidateTip.class, name);
         final DataTreeCandidateNode mockCandidateNode = mock(DataTreeCandidateNode.class, name + "-node");
         doReturn(ModificationType.UNMODIFIED).when(mockCandidateNode).getModificationType();
-        doReturn(YangInstanceIdentifier.EMPTY).when(mockCandidate).getRootPath();
+        doReturn(YangInstanceIdentifier.empty()).when(mockCandidate).getRootPath();
         doReturn(mockCandidateNode).when(mockCandidate).getRootNode();
         return mockCandidate;
     }
 
-    static void commitTransaction(final DataTree store, final DataTreeModification modification)
+    static DataTreeCandidate commitTransaction(final DataTree store, final DataTreeModification modification)
             throws DataValidationFailedException {
         modification.ready();
         store.validate(modification);
-        store.commit(store.prepare(modification));
+        final DataTreeCandidate candidate = store.prepare(modification);
+        store.commit(candidate);
+        return candidate;
     }
 
     @SuppressWarnings("serial")
@@ -489,17 +503,17 @@ public abstract class AbstractShardTest extends AbstractActorTest {
         }
 
         @SuppressWarnings("unchecked")
-        private <T> FutureCallback<T> mockFutureCallback(final FutureCallback<T> actual ) {
+        private static <T> FutureCallback<T> mockFutureCallback(final FutureCallback<T> actual) {
             FutureCallback<T> mock = mock(FutureCallback.class);
             doAnswer(invocation -> {
-                actual.onFailure(invocation.getArgumentAt(0, Throwable.class));
+                actual.onFailure(invocation.getArgument(0));
                 return null;
             }).when(mock).onFailure(any(Throwable.class));
 
             doAnswer(invocation -> {
-                actual.onSuccess((T) invocation.getArgumentAt(0, Throwable.class));
+                actual.onSuccess(invocation.getArgument(0));
                 return null;
-            }).when(mock).onSuccess((T) any(Object.class));
+            }).when(mock).onSuccess((T) nullable(Object.class));
 
             return mock;
         }
@@ -518,5 +532,10 @@ public abstract class AbstractShardTest extends AbstractActorTest {
         public State getState() {
             return delegate.getState();
         }
+
+        @Override
+        Optional<SortedSet<String>> getParticipatingShardNames() {
+            return delegate.getParticipatingShardNames();
+        }
     }
 }