Merge "BUG 3057 - notify added event source by topics created before"
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / test / java / org / opendaylight / controller / cluster / datastore / AbstractTransactionProxyTest.java
index 60625a05fd1e2796136fb146301b69b252700379..a64a5802b8387102bdfac19e4535a440088ccf55 100644 (file)
@@ -23,13 +23,19 @@ import akka.actor.ActorSystem;
 import akka.actor.Props;
 import akka.dispatch.Futures;
 import akka.testkit.JavaTestKit;
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.Timer;
+import com.google.common.base.Objects;
+import com.google.common.base.Optional;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.util.concurrent.CheckedFuture;
 import com.typesafe.config.Config;
 import com.typesafe.config.ConfigFactory;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 import java.util.concurrent.TimeUnit;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -44,17 +50,19 @@ import org.opendaylight.controller.cluster.datastore.TransactionProxy.Transactio
 import org.opendaylight.controller.cluster.datastore.TransactionProxyTest.TestException;
 import org.opendaylight.controller.cluster.datastore.messages.BatchedModifications;
 import org.opendaylight.controller.cluster.datastore.messages.BatchedModificationsReply;
+import org.opendaylight.controller.cluster.datastore.messages.CommitTransactionReply;
 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.PrimaryShardInfo;
 import org.opendaylight.controller.cluster.datastore.messages.ReadData;
 import org.opendaylight.controller.cluster.datastore.messages.ReadDataReply;
-import org.opendaylight.controller.cluster.datastore.messages.ReadyTransaction;
 import org.opendaylight.controller.cluster.datastore.messages.ReadyTransactionReply;
 import org.opendaylight.controller.cluster.datastore.modification.AbstractModification;
 import org.opendaylight.controller.cluster.datastore.modification.Modification;
 import org.opendaylight.controller.cluster.datastore.modification.WriteModification;
 import org.opendaylight.controller.cluster.datastore.shardstrategy.DefaultShardStrategy;
+import org.opendaylight.controller.cluster.datastore.shardstrategy.ShardStrategy;
 import org.opendaylight.controller.cluster.datastore.shardstrategy.ShardStrategyFactory;
 import org.opendaylight.controller.cluster.datastore.utils.ActorContext;
 import org.opendaylight.controller.cluster.datastore.utils.DoNothingActor;
@@ -64,7 +72,10 @@ import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException;
 import org.opendaylight.controller.protobuff.messages.transaction.ShardTransactionMessages.CreateTransactionReply;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
+import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTree;
 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import scala.concurrent.Await;
 import scala.concurrent.Future;
 import scala.concurrent.duration.Duration;
@@ -75,9 +86,28 @@ import scala.concurrent.duration.Duration;
  * @author Thomas Pantelis
  */
 public abstract class AbstractTransactionProxyTest {
+    protected final Logger log = LoggerFactory.getLogger(getClass());
+
     private static ActorSystem system;
 
-    private final Configuration configuration = new MockConfiguration();
+    private final Configuration configuration = new MockConfiguration() {
+        @Override
+        public Map<String, ShardStrategy> getModuleNameToShardStrategyMap() {
+            return ImmutableMap.<String, ShardStrategy>builder().put(
+                    "junk", new ShardStrategy() {
+                        @Override
+                        public String findShard(YangInstanceIdentifier path) {
+                            return "junk";
+                        }
+                    }).build();
+        }
+
+        @Override
+        public Optional<String> getModuleNameFromNameSpace(String nameSpace) {
+            return TestModel.JUNK_QNAME.getNamespace().toASCIIString().equals(nameSpace) ?
+                    Optional.of("junk") : Optional.<String>absent();
+        }
+    };
 
     @Mock
     protected ActorContext mockActorContext;
@@ -89,8 +119,7 @@ public abstract class AbstractTransactionProxyTest {
 
     protected final String memberName = "mock-member";
 
-    protected final Builder dataStoreContextBuilder = DatastoreContext.newBuilder().operationTimeoutInSeconds(2).
-            shardBatchedModificationCount(1);
+    protected final Builder dataStoreContextBuilder = DatastoreContext.newBuilder().operationTimeoutInSeconds(2);
 
     @BeforeClass
     public static void setUpClass() throws IOException {
@@ -123,6 +152,9 @@ public abstract class AbstractTransactionProxyTest {
         doReturn(dataStoreContextBuilder.build()).when(mockActorContext).getDatastoreContext();
         doReturn(10).when(mockActorContext).getTransactionOutstandingOperationLimit();
 
+        Timer timer = new MetricRegistry().timer("test");
+        doReturn(timer).when(mockActorContext).getOperationTimer(any(String.class));
+
         ShardStrategyFactory.setConfiguration(configuration);
     }
 
@@ -200,10 +232,6 @@ public abstract class AbstractTransactionProxyTest {
         return argThat(matcher);
     }
 
-    protected Future<Object> readySerializedTxReply(String path) {
-        return Futures.successful((Object)new ReadyTransactionReply(path).toSerializable());
-    }
-
     protected Future<Object> readyTxReply(String path) {
         return Futures.successful((Object)new ReadyTransactionReply(path));
     }
@@ -222,18 +250,18 @@ public abstract class AbstractTransactionProxyTest {
     }
 
     protected Future<Object> dataExistsSerializedReply(boolean exists) {
-        return Futures.successful(new DataExistsReply(exists).toSerializable());
+        return Futures.successful(DataExistsReply.create(exists).toSerializable());
     }
 
     protected Future<DataExistsReply> dataExistsReply(boolean exists) {
-        return Futures.successful(new DataExistsReply(exists));
+        return Futures.successful(DataExistsReply.create(exists));
     }
 
     protected Future<BatchedModificationsReply> batchedModificationsReply(int count) {
         return Futures.successful(new BatchedModificationsReply(count));
     }
 
-    protected Future<Object> incompleteFuture(){
+    protected Future<Object> incompleteFuture() {
         return mock(Future.class);
     }
 
@@ -246,6 +274,16 @@ public abstract class AbstractTransactionProxyTest {
                 eq(actorSelection(actorRef)), isA(BatchedModifications.class));
     }
 
+    protected void expectBatchedModificationsReady(ActorRef actorRef) {
+        expectBatchedModificationsReady(actorRef, false);
+    }
+
+    protected void expectBatchedModificationsReady(ActorRef actorRef, boolean doCommitOnReady) {
+        doReturn(doCommitOnReady ? Futures.successful(new CommitTransactionReply().toSerializable()) :
+            readyTxReply(actorRef.path().toString())).when(mockActorContext).executeOperationAsync(
+                    eq(actorSelection(actorRef)), isA(BatchedModifications.class));
+    }
+
     protected void expectBatchedModifications(int count) {
         doReturn(batchedModificationsReply(count)).when(mockActorContext).executeOperationAsync(
                 any(ActorSelection.class), isA(BatchedModifications.class));
@@ -256,11 +294,6 @@ public abstract class AbstractTransactionProxyTest {
                 any(ActorSelection.class), isA(BatchedModifications.class));
     }
 
-    protected void expectReadyTransaction(ActorRef actorRef) {
-        doReturn(readySerializedTxReply(actorRef.path().toString())).when(mockActorContext).executeOperationAsync(
-                eq(actorSelection(actorRef)), isA(ReadyTransaction.SERIALIZABLE_CLASS));
-    }
-
     protected void expectFailedBatchedModifications(ActorRef actorRef) {
         doReturn(Futures.failed(new TestException())).when(mockActorContext).executeOperationAsync(
                 eq(actorSelection(actorRef)), isA(BatchedModifications.class));
@@ -275,12 +308,23 @@ public abstract class AbstractTransactionProxyTest {
     }
 
     protected ActorRef setupActorContextWithoutInitialCreateTransaction(ActorSystem actorSystem) {
+        return setupActorContextWithoutInitialCreateTransaction(actorSystem, DefaultShardStrategy.DEFAULT_SHARD);
+    }
+
+    protected Future<PrimaryShardInfo> primaryShardInfoReply(ActorSystem actorSystem, ActorRef actorRef) {
+        return Futures.successful(new PrimaryShardInfo(actorSystem.actorSelection(actorRef.path()),
+                Optional.<DataTree>absent()));
+    }
+
+    protected ActorRef setupActorContextWithoutInitialCreateTransaction(ActorSystem actorSystem, String shardName) {
         ActorRef actorRef = actorSystem.actorOf(Props.create(DoNothingActor.class));
+        log.info("Created mock shard actor {}", actorRef);
+
         doReturn(actorSystem.actorSelection(actorRef.path())).
                 when(mockActorContext).actorSelection(actorRef.path().toString());
 
-        doReturn(Futures.successful(actorSystem.actorSelection(actorRef.path()))).
-                when(mockActorContext).findPrimaryShardAsync(eq(DefaultShardStrategy.DEFAULT_SHARD));
+        doReturn(primaryShardInfoReply(actorSystem, actorRef)).
+                when(mockActorContext).findPrimaryShardAsync(eq(shardName));
 
         doReturn(false).when(mockActorContext).isPathLocal(actorRef.path().toString());
 
@@ -290,20 +334,45 @@ public abstract class AbstractTransactionProxyTest {
     }
 
     protected ActorRef setupActorContextWithInitialCreateTransaction(ActorSystem actorSystem,
-            TransactionType type, int transactionVersion) {
-        ActorRef actorRef = setupActorContextWithoutInitialCreateTransaction(actorSystem);
+            TransactionType type, int transactionVersion, String shardName) {
+        ActorRef shardActorRef = setupActorContextWithoutInitialCreateTransaction(actorSystem, shardName);
 
-        doReturn(Futures.successful(createTransactionReply(actorRef, transactionVersion))).when(mockActorContext).
-                executeOperationAsync(eq(actorSystem.actorSelection(actorRef.path())),
-                        eqCreateTransaction(memberName, type));
+        return setupActorContextWithInitialCreateTransaction(actorSystem, type, transactionVersion,
+                memberName, shardActorRef);
+    }
 
-        return actorRef;
+    protected ActorRef setupActorContextWithInitialCreateTransaction(ActorSystem actorSystem,
+            TransactionType type, int transactionVersion, String prefix, ActorRef shardActorRef) {
+
+        ActorRef txActorRef;
+        if(type == TransactionType.WRITE_ONLY && transactionVersion >= DataStoreVersions.LITHIUM_VERSION &&
+                dataStoreContextBuilder.build().isWriteOnlyTransactionOptimizationsEnabled()) {
+            txActorRef = shardActorRef;
+        } else {
+            txActorRef = actorSystem.actorOf(Props.create(DoNothingActor.class));
+            log.info("Created mock shard Tx actor {}", txActorRef);
+
+            doReturn(actorSystem.actorSelection(txActorRef.path())).
+            when(mockActorContext).actorSelection(txActorRef.path().toString());
+
+            doReturn(Futures.successful(createTransactionReply(txActorRef, transactionVersion))).when(mockActorContext).
+            executeOperationAsync(eq(actorSystem.actorSelection(shardActorRef.path())),
+                    eqCreateTransaction(prefix, type));
+        }
+
+        return txActorRef;
     }
 
     protected ActorRef setupActorContextWithInitialCreateTransaction(ActorSystem actorSystem, TransactionType type) {
-        return setupActorContextWithInitialCreateTransaction(actorSystem, type, DataStoreVersions.CURRENT_VERSION);
+        return setupActorContextWithInitialCreateTransaction(actorSystem, type, DataStoreVersions.CURRENT_VERSION,
+                DefaultShardStrategy.DEFAULT_SHARD);
     }
 
+    protected ActorRef setupActorContextWithInitialCreateTransaction(ActorSystem actorSystem, TransactionType type,
+            String shardName) {
+        return setupActorContextWithInitialCreateTransaction(actorSystem, type, DataStoreVersions.CURRENT_VERSION,
+                shardName);
+    }
 
     protected void propagateReadFailedExceptionCause(CheckedFuture<?, ReadFailedException> future)
             throws Throwable {
@@ -338,17 +407,24 @@ public abstract class AbstractTransactionProxyTest {
         return captured;
     }
 
-    protected void verifyOneBatchedModification(ActorRef actorRef, Modification expected) {
+    protected void verifyOneBatchedModification(ActorRef actorRef, Modification expected, boolean expIsReady) {
         List<BatchedModifications> batchedModifications = captureBatchedModifications(actorRef);
         assertEquals("Captured BatchedModifications count", 1, batchedModifications.size());
 
-        verifyBatchedModifications(batchedModifications.get(0), expected);
+        verifyBatchedModifications(batchedModifications.get(0), expIsReady, expIsReady, expected);
     }
 
-    protected void verifyBatchedModifications(Object message, Modification... expected) {
+    protected void verifyBatchedModifications(Object message, boolean expIsReady, Modification... expected) {
+        verifyBatchedModifications(message, expIsReady, false, expected);
+    }
+
+    protected void verifyBatchedModifications(Object message, boolean expIsReady, boolean expIsDoCommitOnReady,
+            Modification... expected) {
         assertEquals("Message type", BatchedModifications.class, message.getClass());
         BatchedModifications batchedModifications = (BatchedModifications)message;
         assertEquals("BatchedModifications size", expected.length, batchedModifications.getModifications().size());
+        assertEquals("isReady", expIsReady, batchedModifications.isReady());
+        assertEquals("isDoCommitOnReady", expIsDoCommitOnReady, batchedModifications.isDoCommitOnReady());
         for(int i = 0; i < batchedModifications.getModifications().size(); i++) {
             Modification actual = batchedModifications.getModifications().get(i);
             assertEquals("Modification type", expected[i].getClass(), actual.getClass());
@@ -361,27 +437,44 @@ public abstract class AbstractTransactionProxyTest {
         }
     }
 
-    protected void verifyCohortFutures(ThreePhaseCommitCohortProxy proxy,
+    protected void verifyCohortFutures(AbstractThreePhaseCommitCohort<?> proxy,
             Object... expReplies) throws Exception {
             assertEquals("getReadyOperationFutures size", expReplies.length,
                     proxy.getCohortFutures().size());
 
-            int i = 0;
-            for( Future<ActorSelection> future: proxy.getCohortFutures()) {
+            List<Object> futureResults = new ArrayList<>();
+            for( Future<?> future: proxy.getCohortFutures()) {
                 assertNotNull("Ready operation Future is null", future);
+                try {
+                    futureResults.add(Await.result(future, Duration.create(5, TimeUnit.SECONDS)));
+                } catch(Exception e) {
+                    futureResults.add(e);
+                }
+            }
 
-                Object expReply = expReplies[i++];
-                if(expReply instanceof ActorSelection) {
-                    ActorSelection actual = Await.result(future, Duration.create(5, TimeUnit.SECONDS));
-                    assertEquals("Cohort actor path", expReply, actual);
-                } else {
-                    // Expecting exception.
-                    try {
-                        Await.result(future, Duration.create(5, TimeUnit.SECONDS));
-                        fail("Expected exception from ready operation Future");
-                    } catch(Exception e) {
-                        // Expected
+            for(int i = 0; i < expReplies.length; i++) {
+                Object expReply = expReplies[i];
+                boolean found = false;
+                Iterator<?> iter = futureResults.iterator();
+                while(iter.hasNext()) {
+                    Object actual = iter.next();
+                    if(CommitTransactionReply.SERIALIZABLE_CLASS.isInstance(expReply) &&
+                       CommitTransactionReply.SERIALIZABLE_CLASS.isInstance(actual)) {
+                        found = true;
+                    } else if(expReply instanceof ActorSelection && Objects.equal(expReply, actual)) {
+                        found = true;
+                    } else if(expReply instanceof Class && ((Class<?>)expReply).isInstance(actual)) {
+                        found = true;
                     }
+
+                    if(found) {
+                        iter.remove();
+                        break;
+                    }
+                }
+
+                if(!found) {
+                    fail(String.format("No cohort Future response found for %s. Actual: %s", expReply, futureResults));
                 }
             }
         }