BUG 2437 - Enable snapshotting based on size of data in the in-memory journal
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / test / java / org / opendaylight / controller / cluster / datastore / ShardTransactionTest.java
index 36633c55d590c023651a39e82feef8ba48d780d5..f5af93d584ce4ff62678f9f499cd629c96ccc80b 100644 (file)
@@ -1,13 +1,28 @@
 package org.opendaylight.controller.cluster.datastore;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
 import akka.actor.ActorRef;
 import akka.actor.Props;
+import akka.actor.Terminated;
 import akka.testkit.JavaTestKit;
-import com.google.common.util.concurrent.ListeningExecutorService;
+import akka.testkit.TestActorRef;
 import com.google.common.util.concurrent.MoreExecutors;
+import java.util.Collections;
+import java.util.concurrent.TimeUnit;
+import org.junit.BeforeClass;
 import org.junit.Test;
+import org.opendaylight.controller.cluster.datastore.ShardWriteTransaction.GetCompositeModificationReply;
+import org.opendaylight.controller.cluster.datastore.exceptions.UnknownMessageException;
+import org.opendaylight.controller.cluster.datastore.identifiers.ShardIdentifier;
+import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shard.ShardStats;
 import org.opendaylight.controller.cluster.datastore.messages.CloseTransaction;
 import org.opendaylight.controller.cluster.datastore.messages.CloseTransactionReply;
+import org.opendaylight.controller.cluster.datastore.messages.CreateTransaction;
+import org.opendaylight.controller.cluster.datastore.messages.DataExists;
+import org.opendaylight.controller.cluster.datastore.messages.DataExistsReply;
 import org.opendaylight.controller.cluster.datastore.messages.DeleteData;
 import org.opendaylight.controller.cluster.datastore.messages.DeleteDataReply;
 import org.opendaylight.controller.cluster.datastore.messages.MergeData;
@@ -18,219 +33,362 @@ import org.opendaylight.controller.cluster.datastore.messages.ReadyTransaction;
 import org.opendaylight.controller.cluster.datastore.messages.ReadyTransactionReply;
 import org.opendaylight.controller.cluster.datastore.messages.WriteData;
 import org.opendaylight.controller.cluster.datastore.messages.WriteDataReply;
+import org.opendaylight.controller.cluster.datastore.modification.CompositeModification;
+import org.opendaylight.controller.cluster.datastore.modification.DeleteModification;
+import org.opendaylight.controller.cluster.datastore.modification.MergeModification;
+import org.opendaylight.controller.cluster.datastore.modification.Modification;
+import org.opendaylight.controller.cluster.datastore.modification.WriteModification;
 import org.opendaylight.controller.md.cluster.datastore.model.TestModel;
 import org.opendaylight.controller.md.sal.dom.store.impl.InMemoryDOMDataStore;
-import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier;
+import org.opendaylight.controller.protobuff.messages.transaction.ShardTransactionMessages;
+import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
 import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes;
-
-import static org.junit.Assert.assertEquals;
+import org.opendaylight.yangtools.yang.model.api.SchemaContext;
+import scala.concurrent.duration.Duration;
 
 public class ShardTransactionTest extends AbstractActorTest {
-  private static ListeningExecutorService storeExecutor = MoreExecutors.listeningDecorator(MoreExecutors.sameThreadExecutor());
-
-  private static final InMemoryDOMDataStore store = new InMemoryDOMDataStore("OPER", storeExecutor);
-
-  static {
-    store.onGlobalContextUpdated(TestModel.createTestContext());
-  }
-
-  @Test
-  public void testOnReceiveReadData() throws Exception {
-    new JavaTestKit(getSystem()) {{
-      final Props props = ShardTransaction.props(store.newReadWriteTransaction());
-      final ActorRef subject = getSystem().actorOf(props, "testReadData");
-
-      new Within(duration("1 seconds")) {
-        protected void run() {
-
-          subject.tell(new ReadData(InstanceIdentifier.builder().build()), getRef());
-
-          final String out = new ExpectMsg<String>("match hint") {
-            // do not put code outside this method, will run afterwards
-            protected String match(Object in) {
-              if (in instanceof ReadDataReply) {
-                if (((ReadDataReply) in).getNormalizedNode() != null) {
-                  return "match";
-                }
-                return null;
-              } else {
-                throw noMatch();
-              }
-            }
-          }.get(); // this extracts the received message
-
-          assertEquals("match", out);
-
-          expectNoMsg();
-        }
+    private static final InMemoryDOMDataStore store =
+        new InMemoryDOMDataStore("OPER", MoreExecutors.sameThreadExecutor());
 
+    private static final SchemaContext testSchemaContext = TestModel.createTestContext();
 
-      };
-    }};
-  }
+    private static final ShardIdentifier SHARD_IDENTIFIER =
+        ShardIdentifier.builder().memberName("member-1")
+            .shardName("inventory").type("config").build();
 
-  @Test
-  public void testOnReceiveWriteData() throws Exception {
-    new JavaTestKit(getSystem()) {{
-      final Props props = ShardTransaction.props(store.newReadWriteTransaction());
-      final ActorRef subject = getSystem().actorOf(props, "testWriteData");
+    private DatastoreContext datastoreContext = DatastoreContext.newBuilder().build();
 
-      new Within(duration("1 seconds")) {
-        protected void run() {
+    private final ShardStats shardStats = new ShardStats(SHARD_IDENTIFIER.toString(), "DataStore");
 
-          subject.tell(new WriteData(TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME)), getRef());
+    @BeforeClass
+    public static void staticSetup() {
+        store.onGlobalContextUpdated(testSchemaContext);
+    }
 
-          final String out = new ExpectMsg<String>("match hint") {
-            // do not put code outside this method, will run afterwards
-            protected String match(Object in) {
-              if (in instanceof WriteDataReply) {
-                return "match";
-              } else {
-                throw noMatch();
-              }
-            }
-          }.get(); // this extracts the received message
+    private ActorRef createShard(){
+        return getSystem().actorOf(Shard.props(SHARD_IDENTIFIER,
+            Collections.<ShardIdentifier, String>emptyMap(), datastoreContext, TestModel.createTestContext()));
+    }
 
-          assertEquals("match", out);
+    @Test
+    public void testOnReceiveReadData() throws Exception {
+        new JavaTestKit(getSystem()) {{
+            final ActorRef shard = createShard();
+            Props props = ShardTransaction.props(store.newReadOnlyTransaction(), shard,
+                    testSchemaContext, datastoreContext, shardStats, "txn",
+                    CreateTransaction.CURRENT_VERSION);
 
-          expectNoMsg();
+            testOnReceiveReadData(getSystem().actorOf(props, "testReadDataRO"));
+
+            props = ShardTransaction.props(store.newReadWriteTransaction(), shard,
+                    testSchemaContext, datastoreContext, shardStats, "txn",
+                    CreateTransaction.CURRENT_VERSION);
+
+            testOnReceiveReadData(getSystem().actorOf(props, "testReadDataRW"));
         }
 
+        private void testOnReceiveReadData(final ActorRef transaction) {
+            //serialized read
+            transaction.tell(new ReadData(YangInstanceIdentifier.builder().build()).toSerializable(),
+                getRef());
+
+            ShardTransactionMessages.ReadDataReply replySerialized =
+                expectMsgClass(duration("5 seconds"), ReadDataReply.SERIALIZABLE_CLASS);
 
-      };
-    }};
-  }
+            assertNotNull(ReadDataReply.fromSerializable(
+                testSchemaContext,YangInstanceIdentifier.builder().build(), replySerialized)
+                .getNormalizedNode());
 
-  @Test
-  public void testOnReceiveMergeData() throws Exception {
-    new JavaTestKit(getSystem()) {{
-      final Props props = ShardTransaction.props(store.newReadWriteTransaction());
-      final ActorRef subject = getSystem().actorOf(props, "testMergeData");
+            // unserialized read
+            transaction.tell(new ReadData(YangInstanceIdentifier.builder().build()),getRef());
 
-      new Within(duration("1 seconds")) {
-        protected void run() {
+            ReadDataReply reply = expectMsgClass(duration("5 seconds"), ReadDataReply.class);
 
-          subject.tell(new MergeData(TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME)), getRef());
+            assertNotNull(reply.getNormalizedNode());
+        }};
+    }
 
-          final String out = new ExpectMsg<String>("match hint") {
-            // do not put code outside this method, will run afterwards
-            protected String match(Object in) {
-              if (in instanceof MergeDataReply) {
-                return "match";
-              } else {
-                throw noMatch();
-              }
-            }
-          }.get(); // this extracts the received message
+    @Test
+    public void testOnReceiveReadDataWhenDataNotFound() throws Exception {
+        new JavaTestKit(getSystem()) {{
+            final ActorRef shard = createShard();
+            Props props = ShardTransaction.props( store.newReadOnlyTransaction(), shard,
+                    testSchemaContext, datastoreContext, shardStats, "txn",
+                    CreateTransaction.CURRENT_VERSION);
 
-          assertEquals("match", out);
+            testOnReceiveReadDataWhenDataNotFound(getSystem().actorOf(
+                    props, "testReadDataWhenDataNotFoundRO"));
 
-          expectNoMsg();
+            props = ShardTransaction.props( store.newReadWriteTransaction(), shard,
+                    testSchemaContext, datastoreContext, shardStats, "txn",
+                    CreateTransaction.CURRENT_VERSION);
+
+            testOnReceiveReadDataWhenDataNotFound(getSystem().actorOf(
+                    props, "testReadDataWhenDataNotFoundRW"));
         }
 
+        private void testOnReceiveReadDataWhenDataNotFound(final ActorRef transaction) {
+            // serialized read
+            transaction.tell(new ReadData(TestModel.TEST_PATH).toSerializable(), getRef());
+
+            ShardTransactionMessages.ReadDataReply replySerialized =
+                expectMsgClass(duration("5 seconds"), ReadDataReply.SERIALIZABLE_CLASS);
+
+            assertTrue(ReadDataReply.fromSerializable(
+                testSchemaContext, TestModel.TEST_PATH, replySerialized).getNormalizedNode() == null);
 
-      };
-    }};
-  }
+            // unserialized read
+            transaction.tell(new ReadData(TestModel.TEST_PATH),getRef());
 
-  @Test
-  public void testOnReceiveDeleteData() throws Exception {
-    new JavaTestKit(getSystem()) {{
-      final Props props = ShardTransaction.props(store.newReadWriteTransaction());
-      final ActorRef subject = getSystem().actorOf(props, "testDeleteData");
+            ReadDataReply reply = expectMsgClass(duration("5 seconds"), ReadDataReply.class);
 
-      new Within(duration("1 seconds")) {
-        protected void run() {
+            assertTrue(reply.getNormalizedNode() == null);
+        }};
+    }
 
-          subject.tell(new DeleteData(TestModel.TEST_PATH), getRef());
+    @Test
+    public void testOnReceiveDataExistsPositive() throws Exception {
+        new JavaTestKit(getSystem()) {{
+            final ActorRef shard = createShard();
+            Props props = ShardTransaction.props(store.newReadOnlyTransaction(), shard,
+                    testSchemaContext, datastoreContext, shardStats, "txn",
+                    CreateTransaction.CURRENT_VERSION);
 
-          final String out = new ExpectMsg<String>("match hint") {
-            // do not put code outside this method, will run afterwards
-            protected String match(Object in) {
-              if (in instanceof DeleteDataReply) {
-                return "match";
-              } else {
-                throw noMatch();
-              }
-            }
-          }.get(); // this extracts the received message
+            testOnReceiveDataExistsPositive(getSystem().actorOf(props, "testDataExistsPositiveRO"));
 
-          assertEquals("match", out);
+            props = ShardTransaction.props(store.newReadWriteTransaction(), shard,
+                    testSchemaContext, datastoreContext, shardStats, "txn",
+                    CreateTransaction.CURRENT_VERSION);
 
-          expectNoMsg();
+            testOnReceiveDataExistsPositive(getSystem().actorOf(props, "testDataExistsPositiveRW"));
         }
 
+        private void testOnReceiveDataExistsPositive(final ActorRef transaction) {
+            transaction.tell(new DataExists(YangInstanceIdentifier.builder().build()).toSerializable(),
+                getRef());
 
-      };
-    }};
-  }
+            ShardTransactionMessages.DataExistsReply replySerialized =
+                expectMsgClass(duration("5 seconds"), ShardTransactionMessages.DataExistsReply.class);
 
+            assertTrue(DataExistsReply.fromSerializable(replySerialized).exists());
 
-  @Test
-  public void testOnReceiveReadyTransaction() throws Exception {
-    new JavaTestKit(getSystem()) {{
-      final Props props = ShardTransaction.props(store.newReadWriteTransaction());
-      final ActorRef subject = getSystem().actorOf(props, "testReadyTransaction");
+            // unserialized read
+            transaction.tell(new DataExists(YangInstanceIdentifier.builder().build()),getRef());
 
-      new Within(duration("1 seconds")) {
-        protected void run() {
+            DataExistsReply reply = expectMsgClass(duration("5 seconds"), DataExistsReply.class);
 
-          subject.tell(new ReadyTransaction(), getRef());
+            assertTrue(reply.exists());
+        }};
+    }
 
-          final String out = new ExpectMsg<String>("match hint") {
-            // do not put code outside this method, will run afterwards
-            protected String match(Object in) {
-              if (in instanceof ReadyTransactionReply) {
-                return "match";
-              } else {
-                throw noMatch();
-              }
-            }
-          }.get(); // this extracts the received message
+    @Test
+    public void testOnReceiveDataExistsNegative() throws Exception {
+        new JavaTestKit(getSystem()) {{
+            final ActorRef shard = createShard();
+            Props props = ShardTransaction.props(store.newReadOnlyTransaction(), shard,
+                    testSchemaContext, datastoreContext, shardStats, "txn",
+                    CreateTransaction.CURRENT_VERSION);
 
-          assertEquals("match", out);
+            testOnReceiveDataExistsNegative(getSystem().actorOf(props, "testDataExistsNegativeRO"));
 
-          expectNoMsg();
+            props = ShardTransaction.props(store.newReadWriteTransaction(), shard,
+                    testSchemaContext, datastoreContext, shardStats, "txn",
+                    CreateTransaction.CURRENT_VERSION);
+
+            testOnReceiveDataExistsNegative(getSystem().actorOf(props, "testDataExistsNegativeRW"));
         }
 
+        private void testOnReceiveDataExistsNegative(final ActorRef transaction) {
+            transaction.tell(new DataExists(TestModel.TEST_PATH).toSerializable(), getRef());
 
-      };
-    }};
+            ShardTransactionMessages.DataExistsReply replySerialized =
+                expectMsgClass(duration("5 seconds"), ShardTransactionMessages.DataExistsReply.class);
 
-  }
+            assertFalse(DataExistsReply.fromSerializable(replySerialized).exists());
 
-  @Test
-  public void testOnReceiveCloseTransaction() throws Exception {
-    new JavaTestKit(getSystem()) {{
-      final Props props = ShardTransaction.props(store.newReadWriteTransaction());
-      final ActorRef subject = getSystem().actorOf(props, "testCloseTransaction");
+            // unserialized read
+            transaction.tell(new DataExists(TestModel.TEST_PATH),getRef());
 
-      new Within(duration("1 seconds")) {
-        protected void run() {
+            DataExistsReply reply = expectMsgClass(duration("5 seconds"), DataExistsReply.class);
 
-          subject.tell(new CloseTransaction(), getRef());
+            assertFalse(reply.exists());
+        }};
+    }
 
-          final String out = new ExpectMsg<String>("match hint") {
-            // do not put code outside this method, will run afterwards
-            protected String match(Object in) {
-              if (in instanceof CloseTransactionReply) {
-                return "match";
-              } else {
-                throw noMatch();
-              }
-            }
-          }.get(); // this extracts the received message
+    private void assertModification(final ActorRef subject,
+        final Class<? extends Modification> modificationType) {
+        new JavaTestKit(getSystem()) {{
+            subject.tell(new ShardWriteTransaction.GetCompositedModification(), getRef());
 
-          assertEquals("match", out);
+            CompositeModification compositeModification = expectMsgClass(duration("3 seconds"),
+                    GetCompositeModificationReply.class).getModification();
 
-          expectNoMsg();
-        }
+            assertTrue(compositeModification.getModifications().size() == 1);
+            assertEquals(modificationType, compositeModification.getModifications().get(0).getClass());
+        }};
+    }
+
+    @Test
+    public void testOnReceiveWriteData() throws Exception {
+        new JavaTestKit(getSystem()) {{
+            final ActorRef shard = createShard();
+            final Props props = ShardTransaction.props(store.newWriteOnlyTransaction(), shard,
+                    testSchemaContext, datastoreContext, shardStats, "txn",
+                    CreateTransaction.CURRENT_VERSION);
+            final ActorRef transaction = getSystem().actorOf(props, "testWriteData");
+
+            transaction.tell(new WriteData(TestModel.TEST_PATH,
+                ImmutableNodes.containerNode(TestModel.TEST_QNAME), TestModel.createTestContext()).toSerializable(),
+                getRef());
+
+            expectMsgClass(duration("5 seconds"), ShardTransactionMessages.WriteDataReply.class);
+
+            assertModification(transaction, WriteModification.class);
+
+            //unserialized write
+            transaction.tell(new WriteData(TestModel.TEST_PATH,
+                ImmutableNodes.containerNode(TestModel.TEST_QNAME),
+                TestModel.createTestContext()),
+                getRef());
+
+            expectMsgClass(duration("5 seconds"), WriteDataReply.class);
+        }};
+    }
+
+    @Test
+    public void testOnReceiveMergeData() throws Exception {
+        new JavaTestKit(getSystem()) {{
+            final ActorRef shard = createShard();
+            final Props props = ShardTransaction.props(store.newReadWriteTransaction(), shard,
+                    testSchemaContext, datastoreContext, shardStats, "txn",
+                    CreateTransaction.CURRENT_VERSION);
+            final ActorRef transaction = getSystem().actorOf(props, "testMergeData");
+
+            transaction.tell(new MergeData(TestModel.TEST_PATH,
+                ImmutableNodes.containerNode(TestModel.TEST_QNAME), testSchemaContext).toSerializable(),
+                getRef());
+
+            expectMsgClass(duration("5 seconds"), ShardTransactionMessages.MergeDataReply.class);
+
+            assertModification(transaction, MergeModification.class);
+
+            //unserialized merge
+            transaction.tell(new MergeData(TestModel.TEST_PATH,
+                ImmutableNodes.containerNode(TestModel.TEST_QNAME), testSchemaContext),
+                getRef());
+
+            expectMsgClass(duration("5 seconds"), MergeDataReply.class);
+        }};
+    }
+
+    @Test
+    public void testOnReceiveDeleteData() throws Exception {
+        new JavaTestKit(getSystem()) {{
+            final ActorRef shard = createShard();
+            final Props props = ShardTransaction.props( store.newWriteOnlyTransaction(), shard,
+                    testSchemaContext, datastoreContext, shardStats, "txn",
+                    CreateTransaction.CURRENT_VERSION);
+            final ActorRef transaction = getSystem().actorOf(props, "testDeleteData");
+
+            transaction.tell(new DeleteData(TestModel.TEST_PATH).toSerializable(), getRef());
 
+            expectMsgClass(duration("5 seconds"), ShardTransactionMessages.DeleteDataReply.class);
 
-      };
-    }};
+            assertModification(transaction, DeleteModification.class);
 
-  }
+            //unserialized merge
+            transaction.tell(new DeleteData(TestModel.TEST_PATH), getRef());
+
+            expectMsgClass(duration("5 seconds"), DeleteDataReply.class);
+        }};
+    }
+
+
+    @Test
+    public void testOnReceiveReadyTransaction() throws Exception {
+        new JavaTestKit(getSystem()) {{
+            final ActorRef shard = createShard();
+            final Props props = ShardTransaction.props( store.newReadWriteTransaction(), shard,
+                    testSchemaContext, datastoreContext, shardStats, "txn",
+                    CreateTransaction.CURRENT_VERSION);
+            final ActorRef transaction = getSystem().actorOf(props, "testReadyTransaction");
+
+            watch(transaction);
+
+            transaction.tell(new ReadyTransaction().toSerializable(), getRef());
+
+            expectMsgAnyClassOf(duration("5 seconds"), ReadyTransactionReply.SERIALIZABLE_CLASS,
+                    Terminated.class);
+            expectMsgAnyClassOf(duration("5 seconds"), ReadyTransactionReply.SERIALIZABLE_CLASS,
+                    Terminated.class);
+        }};
+
+        // test
+        new JavaTestKit(getSystem()) {{
+            final ActorRef shard = createShard();
+            final Props props = ShardTransaction.props( store.newReadWriteTransaction(), shard,
+                testSchemaContext, datastoreContext, shardStats, "txn",
+                CreateTransaction.CURRENT_VERSION);
+            final ActorRef transaction = getSystem().actorOf(props, "testReadyTransaction2");
+
+            watch(transaction);
+
+            transaction.tell(new ReadyTransaction(), getRef());
+
+            expectMsgAnyClassOf(duration("5 seconds"), ReadyTransactionReply.class,
+                    Terminated.class);
+            expectMsgAnyClassOf(duration("5 seconds"), ReadyTransactionReply.class,
+                    Terminated.class);
+        }};
+
+    }
+
+    @Test
+    public void testOnReceiveCloseTransaction() throws Exception {
+        new JavaTestKit(getSystem()) {{
+            final ActorRef shard = createShard();
+            final Props props = ShardTransaction.props(store.newReadWriteTransaction(), shard,
+                    testSchemaContext, datastoreContext, shardStats, "txn",
+                    CreateTransaction.CURRENT_VERSION);
+            final ActorRef transaction = getSystem().actorOf(props, "testCloseTransaction");
+
+            watch(transaction);
+
+            transaction.tell(new CloseTransaction().toSerializable(), getRef());
+
+            expectMsgClass(duration("3 seconds"), CloseTransactionReply.SERIALIZABLE_CLASS);
+            expectTerminated(duration("3 seconds"), transaction);
+        }};
+    }
+
+    @Test(expected=UnknownMessageException.class)
+    public void testNegativePerformingWriteOperationOnReadTransaction() throws Exception {
+        final ActorRef shard = createShard();
+        final Props props = ShardTransaction.props(store.newReadOnlyTransaction(), shard,
+                testSchemaContext, datastoreContext, shardStats, "txn",
+                CreateTransaction.CURRENT_VERSION);
+        final TestActorRef<ShardTransaction> transaction = TestActorRef.apply(props,getSystem());
+
+        transaction.receive(new DeleteData(TestModel.TEST_PATH).toSerializable(), ActorRef.noSender());
+    }
+
+    @Test
+    public void testShardTransactionInactivity() {
+
+        datastoreContext = DatastoreContext.newBuilder().shardTransactionIdleTimeout(
+                Duration.create(500, TimeUnit.MILLISECONDS)).build();
+
+        new JavaTestKit(getSystem()) {{
+            final ActorRef shard = createShard();
+            final Props props = ShardTransaction.props(store.newReadWriteTransaction(), shard,
+                    testSchemaContext, datastoreContext, shardStats, "txn",
+                    CreateTransaction.CURRENT_VERSION);
+            final ActorRef transaction =
+                getSystem().actorOf(props, "testShardTransactionInactivity");
 
+            watch(transaction);
 
-}
\ No newline at end of file
+            expectMsgClass(duration("3 seconds"), Terminated.class);
+        }};
+    }
+}