Merge "Updated RpcRegistry to accept a list of route identifiers while adding or...
authorMoiz Raja <moraja@cisco.com>
Tue, 12 Aug 2014 15:12:24 +0000 (15:12 +0000)
committerGerrit Code Review <gerrit@opendaylight.org>
Tue, 12 Aug 2014 15:12:24 +0000 (15:12 +0000)
13 files changed:
opendaylight/md-sal/sal-clustering-config/src/main/resources/initial/akka.conf
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/Shard.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardReadTransaction.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardReadWriteTransaction.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardTransaction.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardWriteTransaction.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/jmx/mbeans/shard/ShardMBeanFactory.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/jmx/mbeans/shard/ShardStats.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/jmx/mbeans/shard/ShardStatsMBean.java
opendaylight/md-sal/sal-distributed-datastore/src/main/resources/application.conf
opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTransactionFailureTest.java [new file with mode: 0644]
opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/jmx/mbeans/shard/ShardStatsTest.java
opendaylight/md-sal/sal-rest-connector-config/src/main/resources/initial/10-rest-connector.xml

index 9749ae27ae671559c4fb5b08c85620ece27188a3..5bf231dbe1ca31040565bad6b82d90201a4c0df2 100644 (file)
@@ -18,9 +18,9 @@ odl-cluster-data {
       netty.tcp {
         hostname = "<CHANGE_ME>"
         port = 2550
-           maximum-frame-size = 2097152
-           send-buffer-size = 52428800
-           receive-buffer-size = 52428800
+        maximum-frame-size = 419430400
+        send-buffer-size = 52428800
+        receive-buffer-size = 52428800
       }
     }
 
index 27744fcb8de8e5cde22c16090908850f231bfec8..21fea96320f30754baa2c300877c768c895b4d02 100644 (file)
@@ -48,6 +48,7 @@ import org.opendaylight.yangtools.yang.model.api.SchemaContext;
 import scala.concurrent.duration.FiniteDuration;
 
 import java.util.ArrayList;
+import java.util.Date;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -143,7 +144,7 @@ public class Shard extends RaftActor {
         } else if (message instanceof PeerAddressResolved) {
             PeerAddressResolved resolved = (PeerAddressResolved) message;
             setPeerAddress(resolved.getPeerId(), resolved.getPeerAddress());
-        } else {
+        } else{
             super.onReceiveCommand(message);
         }
     }
@@ -190,16 +191,14 @@ public class Shard extends RaftActor {
 
         getSender()
             .tell(new CreateTransactionReply(
-                    Serialization.serializedActorPath(transactionActor),
-                    createTransaction.getTransactionId()).toSerializable(),
-                getSelf()
-            );
+                Serialization.serializedActorPath(transactionActor),
+                createTransaction.getTransactionId()).toSerializable(),
+                getSelf());
     }
 
     private void commit(final ActorRef sender, Object serialized) {
-        Modification modification =
-            MutableCompositeModification.fromSerializable(
-                serialized, schemaContext);
+        Modification modification = MutableCompositeModification
+            .fromSerializable(serialized, schemaContext);
         DOMStoreThreePhaseCommitCohort cohort =
             modificationToCohort.remove(serialized);
         if (cohort == null) {
@@ -216,32 +215,32 @@ public class Shard extends RaftActor {
                 future.get();
                 future = commitCohort.commit();
                 future.get();
-            } catch (InterruptedException e) {
-                LOG.error("Failed to commit", e);
-            } catch (ExecutionException e) {
+            } catch (InterruptedException | ExecutionException e) {
+                shardMBean.incrementFailedTransactionsCount();
                 LOG.error("Failed to commit", e);
+                return;
             }
+            //we want to just apply the recovery commit and return
+            shardMBean.incrementCommittedTransactionCount();
+            return;
         }
 
         final ListenableFuture<Void> future = cohort.commit();
-        shardMBean.incrementCommittedTransactionCount();
         final ActorRef self = getSelf();
         future.addListener(new Runnable() {
             @Override
             public void run() {
                 try {
                     future.get();
-
-                    if (sender != null) {
                         sender
                             .tell(new CommitTransactionReply().toSerializable(),
                                 self);
-                    } else {
-                        LOG.error("sender is null ???");
-                    }
+                        shardMBean.incrementCommittedTransactionCount();
+                        shardMBean.setLastCommittedTransactionTime(new Date());
+
                 } catch (InterruptedException | ExecutionException e) {
-                    // FIXME : Handle this properly
-                    LOG.error(e, "An exception happened when committing");
+                    shardMBean.incrementFailedTransactionsCount();
+                    sender.tell(new akka.actor.Status.Failure(e),self);
                 }
             }
         }, getContext().dispatcher());
@@ -318,8 +317,7 @@ public class Shard extends RaftActor {
         getSender()
             .tell(new CreateTransactionChainReply(transactionChain.path())
                     .toSerializable(),
-                getSelf()
-            );
+                getSelf());
     }
 
     @Override protected void applyState(ActorRef clientActor, String identifier,
index f78935b5e72b6c79070efe5b8cbd315fa535ac12..7a18fca100f027b2bfad23b0433eeb91eab06484 100644 (file)
@@ -55,4 +55,9 @@ public class ShardReadTransaction extends ShardTransaction {
     getSelf().tell(PoisonPill.getInstance(), getSelf());
   }
 
+  //default scope test method to check if we get correct exception
+  void forUnitTestOnlyExplicitTransactionClose(){
+      transaction.close();
+  }
+
 }
index 6733bcfb9f6e323be99a849195bddac637f11588..92fb39308357cd63acf2d7810a321b1081b3ace1 100644 (file)
@@ -65,4 +65,13 @@ public class ShardReadWriteTransaction extends ShardTransaction {
     getSender().tell(new CloseTransactionReply().toSerializable(), getSelf());
     getSelf().tell(PoisonPill.getInstance(), getSelf());
   }
+
+    /**
+     * The following method is used in unit testing only
+     * hence the default scope.
+     * This is done to test out failure cases.
+     */
+    public void forUnitTestOnlyExplicitTransactionClose() {
+        transaction.close();
+    }
 }
index 3a916bda2c6e163d75c4d39a0c4d80b94b823fb2..312ec9a4ff97f861b1fcbc23220dc44e6920b61a 100644 (file)
@@ -32,6 +32,7 @@ import org.opendaylight.controller.cluster.datastore.modification.ImmutableCompo
 import org.opendaylight.controller.cluster.datastore.modification.MergeModification;
 import org.opendaylight.controller.cluster.datastore.modification.MutableCompositeModification;
 import org.opendaylight.controller.cluster.datastore.modification.WriteModification;
+import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadTransaction;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadWriteTransaction;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCohort;
@@ -41,8 +42,6 @@ import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
 
-import java.util.concurrent.ExecutionException;
-
 /**
  * The ShardTransaction Actor represents a remote transaction
  * <p>
@@ -197,10 +196,9 @@ public abstract class ShardTransaction extends AbstractUntypedActor {
           } else {
             sender.tell(new ReadDataReply(schemaContext,null).toSerializable(), self);
           }
-        } catch (InterruptedException | ExecutionException e) {
-          log.error(e,
-              "An exception happened when reading data from path : "
-                  + path.toString());
+        } catch (Exception e) {
+            sender.tell(new akka.actor.Status.Failure(new ReadFailedException( "An Exception occurred  when reading data from path : "
+                + path.toString(),e)),self);
         }
 
       }
@@ -212,22 +210,35 @@ public abstract class ShardTransaction extends AbstractUntypedActor {
     modification.addModification(
         new WriteModification(message.getPath(), message.getData(),schemaContext));
     LOG.debug("writeData at path : " + message.getPath().toString());
-    transaction.write(message.getPath(), message.getData());
-    getSender().tell(new WriteDataReply().toSerializable(), getSelf());
+
+    try {
+        transaction.write(message.getPath(), message.getData());
+        getSender().tell(new WriteDataReply().toSerializable(), getSelf());
+    }catch(Exception e){
+        getSender().tell(new akka.actor.Status.Failure(e), getSelf());
+    }
   }
 
   protected void mergeData(DOMStoreWriteTransaction transaction, MergeData message) {
     modification.addModification(
         new MergeModification(message.getPath(), message.getData(), schemaContext));
     LOG.debug("mergeData at path : " + message.getPath().toString());
-    transaction.merge(message.getPath(), message.getData());
-    getSender().tell(new MergeDataReply().toSerializable(), getSelf());
+    try {
+        transaction.merge(message.getPath(), message.getData());
+        getSender().tell(new MergeDataReply().toSerializable(), getSelf());
+    }catch(Exception e){
+        getSender().tell(new akka.actor.Status.Failure(e), getSelf());
+    }
   }
 
   protected void deleteData(DOMStoreWriteTransaction transaction, DeleteData message) {
     modification.addModification(new DeleteModification(message.getPath()));
-    transaction.delete(message.getPath());
-    getSender().tell(new DeleteDataReply().toSerializable(), getSelf());
+    try {
+        transaction.delete(message.getPath());
+        getSender().tell(new DeleteDataReply().toSerializable(), getSelf());
+    }catch(Exception e){
+        getSender().tell(new akka.actor.Status.Failure(e), getSelf());
+    }
   }
 
   protected void readyTransaction(DOMStoreWriteTransaction transaction, ReadyTransaction message) {
index 2a5429ba81c4b0025e0799631da1ba914956ac3c..91e578b46d1f6de8f2e841f7d7f459541a9c3bdb 100644 (file)
@@ -63,4 +63,13 @@ public class ShardWriteTransaction extends ShardTransaction {
     getSender().tell(new CloseTransactionReply().toSerializable(), getSelf());
     getSelf().tell(PoisonPill.getInstance(), getSelf());
   }
+
+    /**
+     * The following method is used in unit testing only
+     * hence the default scope.
+     * This is done to test out failure cases.
+     */
+    public void forUnitTestOnlyExplicitTransactionClose() {
+        transaction.close();
+    }
 }
index a3359086b6efdc6eea954eeb6168eb1edf906ff8..afca87f0df6458a041b8d2219e3e578066513481 100644 (file)
@@ -8,19 +8,20 @@ import java.util.Map;
  * Date: 7/16/14
  */
 public class ShardMBeanFactory {
-  private static Map<String,ShardStats> shardMBeans= new HashMap<String,ShardStats>();
+    private static Map<String, ShardStats> shardMBeans =
+        new HashMap<String, ShardStats>();
 
-  public static ShardStats getShardStatsMBean(String shardName){
-       if(shardMBeans.containsKey(shardName)){
+    public static ShardStats getShardStatsMBean(String shardName) {
+        if (shardMBeans.containsKey(shardName)) {
             return shardMBeans.get(shardName);
-       }else {
-         ShardStats shardStatsMBeanImpl = new ShardStats(shardName);
+        } else {
+            ShardStats shardStatsMBeanImpl = new ShardStats(shardName);
 
-         if(shardStatsMBeanImpl.registerMBean()) {
-           shardMBeans.put(shardName, shardStatsMBeanImpl);
-         }
-         return shardStatsMBeanImpl;
-       }
-  }
+            if (shardStatsMBeanImpl.registerMBean()) {
+                shardMBeans.put(shardName, shardStatsMBeanImpl);
+            }
+            return shardStatsMBeanImpl;
+        }
+    }
 
 }
index 09bf00ddfd6196523c87f2cb72749d5ff9b67150..c6c1579ce336dc78e64ecc6b79042c30eee28148 100644 (file)
@@ -2,6 +2,9 @@ package org.opendaylight.controller.cluster.datastore.jmx.mbeans.shard;
 
 import org.opendaylight.controller.cluster.datastore.jmx.mbeans.AbstractBaseMBean;
 
+import java.text.SimpleDateFormat;
+import java.util.Date;
+
 /**
  * @author: syedbahm
  */
@@ -31,6 +34,13 @@ public class ShardStats extends AbstractBaseMBean implements ShardStatsMBean {
 
     private Long lastApplied = -1L;
 
+    private Date lastCommittedTransactionTime = new Date(0L);
+
+    private Long failedTransactionsCount = 0L;
+
+    private SimpleDateFormat sdf =
+        new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS");
+
     ShardStats(String shardName) {
         this.shardName = shardName;
     }
@@ -86,6 +96,16 @@ public class ShardStats extends AbstractBaseMBean implements ShardStatsMBean {
         return lastApplied;
     }
 
+    @Override
+    public String getLastCommittedTransactionTime() {
+
+        return sdf.format(lastCommittedTransactionTime);
+    }
+
+    @Override public Long getFailedTransactionsCount() {
+        return failedTransactionsCount;
+    }
+
     public Long incrementCommittedTransactionCount() {
         return committedTransactionsCount++;
     }
@@ -130,6 +150,12 @@ public class ShardStats extends AbstractBaseMBean implements ShardStatsMBean {
         this.lastApplied = lastApplied;
     }
 
+
+    public void setLastCommittedTransactionTime(
+        Date lastCommittedTransactionTime) {
+        this.lastCommittedTransactionTime = lastCommittedTransactionTime;
+    }
+
     @Override
     protected String getMBeanName() {
         return shardName;
@@ -146,4 +172,7 @@ public class ShardStats extends AbstractBaseMBean implements ShardStatsMBean {
     }
 
 
+    public void incrementFailedTransactionsCount() {
+        this.failedTransactionsCount++;
+    }
 }
index daac89c4c8adaef39047034138a8318e639f54c8..6178f4903e734a88fe12a85b1dcf0972d68a12b7 100644 (file)
@@ -23,7 +23,7 @@ odl-cluster-data {
       netty.tcp {
         hostname = "127.0.0.1"
         port = 2550
-           maximum-frame-size = 2097152
+           maximum-frame-size = 419430400
            send-buffer-size = 52428800
            receive-buffer-size = 52428800
       }
diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTransactionFailureTest.java b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTransactionFailureTest.java
new file mode 100644 (file)
index 0000000..86016a6
--- /dev/null
@@ -0,0 +1,303 @@
+/*
+ *
+ *  Copyright (c) 2014 Cisco Systems, Inc. and others.  All rights reserved.
+ *
+ *  This program and the accompanying materials are made available under the
+ *  terms of the Eclipse Public License v1.0 which accompanies this distribution,
+ *  and is available at http://www.eclipse.org/legal/epl-v10.html
+ *
+ */
+
+package org.opendaylight.controller.cluster.datastore;
+
+import akka.actor.ActorRef;
+import akka.actor.Props;
+import akka.testkit.TestActorRef;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import org.junit.Assert;
+import org.junit.Test;
+import org.opendaylight.controller.cluster.datastore.messages.DeleteData;
+import org.opendaylight.controller.md.cluster.datastore.model.TestModel;
+import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException;
+import org.opendaylight.controller.md.sal.dom.store.impl.InMemoryDOMDataStore;
+import org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages;
+import org.opendaylight.controller.protobuff.messages.transaction.ShardTransactionMessages;
+import org.opendaylight.yangtools.yang.model.api.SchemaContext;
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.Duration;
+
+import java.util.Collections;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Covers negative test cases
+ * @author Basheeruddin Ahmed <syedbahm@cisco.com>
+ */
+public class ShardTransactionFailureTest extends AbstractActorTest {
+    private static ListeningExecutorService storeExecutor =
+        MoreExecutors.listeningDecorator(MoreExecutors.sameThreadExecutor());
+
+    private static final InMemoryDOMDataStore store =
+        new InMemoryDOMDataStore("OPER", storeExecutor,
+            MoreExecutors.sameThreadExecutor());
+
+    private static final SchemaContext testSchemaContext =
+        TestModel.createTestContext();
+
+    static {
+        store.onGlobalContextUpdated(testSchemaContext);
+    }
+
+
+    @Test
+    public void testNegativePerformingWriteOperationOnReadTransaction()
+        throws Exception {
+        try {
+
+            final ActorRef
+                shard = getSystem()
+                .actorOf(Shard.props("config", Collections.EMPTY_MAP));
+            final Props props =
+                ShardTransaction
+                    .props(store.newReadOnlyTransaction(), shard, TestModel
+                        .createTestContext());
+            final TestActorRef subject = TestActorRef.apply(props, getSystem());
+
+            subject
+                .receive(new DeleteData(TestModel.TEST_PATH).toSerializable(),
+                    ActorRef.noSender());
+            Assert.assertFalse(true);
+
+
+        } catch (Exception cs) {
+            assertEquals(cs.getClass().getSimpleName(),
+                Exception.class.getSimpleName());
+            assertTrue(cs.getMessage().startsWith(
+                "ShardTransaction:handleRecieve received an unknown message"));
+        }
+    }
+
+    @Test(expected = ReadFailedException.class)
+    public void testNegativeReadWithReadOnlyTransactionClosed()
+        throws Throwable {
+
+        final ActorRef shard =
+            getSystem().actorOf(Shard.props("config", Collections.EMPTY_MAP));
+        final Props props =
+            ShardTransaction.props(store.newReadOnlyTransaction(), shard,
+                TestModel.createTestContext());
+
+        final TestActorRef<ShardTransaction> subject = TestActorRef
+            .create(getSystem(), props,
+                "testNegativeReadWithReadOnlyTransactionClosed");
+
+        ShardTransactionMessages.ReadData readData =
+            ShardTransactionMessages.ReadData.newBuilder()
+                .setInstanceIdentifierPathArguments(
+                    NormalizedNodeMessages.InstanceIdentifier.newBuilder()
+                        .build()
+                ).build();
+        Future<Object> future =
+            akka.pattern.Patterns.ask(subject, readData, 3000);
+        assertTrue(future.isCompleted());
+        Await.result(future, Duration.Zero());
+
+        ((ShardReadTransaction) subject.underlyingActor())
+            .forUnitTestOnlyExplicitTransactionClose();
+
+        future = akka.pattern.Patterns.ask(subject, readData, 3000);
+        Await.result(future, Duration.Zero());
+
+
+    }
+
+
+    @Test(expected = ReadFailedException.class)
+    public void testNegativeReadWithReadWriteOnlyTransactionClosed()
+        throws Throwable {
+
+        final ActorRef shard =
+            getSystem().actorOf(Shard.props("config", Collections.EMPTY_MAP));
+        final Props props =
+            ShardTransaction.props(store.newReadWriteTransaction(), shard,
+                TestModel.createTestContext());
+
+        final TestActorRef<ShardTransaction> subject = TestActorRef
+            .create(getSystem(), props,
+                "testNegativeReadWithReadWriteOnlyTransactionClosed");
+
+        ShardTransactionMessages.ReadData readData =
+            ShardTransactionMessages.ReadData.newBuilder()
+                .setInstanceIdentifierPathArguments(
+                    NormalizedNodeMessages.InstanceIdentifier.newBuilder()
+                        .build()
+                ).build();
+        Future<Object> future =
+            akka.pattern.Patterns.ask(subject, readData, 3000);
+        assertTrue(future.isCompleted());
+        Await.result(future, Duration.Zero());
+
+        ((ShardReadWriteTransaction) subject.underlyingActor())
+            .forUnitTestOnlyExplicitTransactionClose();
+
+        future = akka.pattern.Patterns.ask(subject, readData, 3000);
+        Await.result(future, Duration.Zero());
+
+
+    }
+
+
+    @Test(expected = IllegalStateException.class)
+    public void testNegativeWriteWithTransactionReady() throws Exception {
+
+
+        final ActorRef shard =
+            getSystem().actorOf(Shard.props("config", Collections.EMPTY_MAP));
+        final Props props =
+            ShardTransaction.props(store.newWriteOnlyTransaction(), shard,
+                TestModel.createTestContext());
+
+        final TestActorRef<ShardTransaction> subject = TestActorRef
+            .create(getSystem(), props,
+                "testNegativeWriteWithTransactionReady");
+
+        ShardTransactionMessages.ReadyTransaction readyTransaction =
+            ShardTransactionMessages.ReadyTransaction.newBuilder().build();
+
+        Future<Object> future =
+            akka.pattern.Patterns.ask(subject, readyTransaction, 3000);
+        assertTrue(future.isCompleted());
+        Await.result(future, Duration.Zero());
+
+        ShardTransactionMessages.WriteData writeData =
+            ShardTransactionMessages.WriteData.newBuilder()
+                .setInstanceIdentifierPathArguments(
+                    NormalizedNodeMessages.InstanceIdentifier.newBuilder()
+                        .build()).setNormalizedNode(
+                NormalizedNodeMessages.Node.newBuilder().build()
+
+            ).build();
+
+        future = akka.pattern.Patterns.ask(subject, writeData, 3000);
+        assertTrue(future.isCompleted());
+        Await.result(future, Duration.Zero());
+
+
+    }
+
+
+    @Test(expected = IllegalStateException.class)
+    public void testNegativeReadWriteWithTransactionReady() throws Exception {
+
+
+        final ActorRef shard =
+            getSystem().actorOf(Shard.props("config", Collections.EMPTY_MAP));
+        final Props props =
+            ShardTransaction.props(store.newReadWriteTransaction(), shard,
+                TestModel.createTestContext());
+
+        final TestActorRef<ShardTransaction> subject = TestActorRef
+            .create(getSystem(), props,
+                "testNegativeReadWriteWithTransactionReady");
+
+        ShardTransactionMessages.ReadyTransaction readyTransaction =
+            ShardTransactionMessages.ReadyTransaction.newBuilder().build();
+
+        Future<Object> future =
+            akka.pattern.Patterns.ask(subject, readyTransaction, 3000);
+        assertTrue(future.isCompleted());
+        Await.result(future, Duration.Zero());
+
+        ShardTransactionMessages.WriteData writeData =
+            ShardTransactionMessages.WriteData.newBuilder()
+                .setInstanceIdentifierPathArguments(
+                    NormalizedNodeMessages.InstanceIdentifier.newBuilder()
+                        .build()).setNormalizedNode(
+                NormalizedNodeMessages.Node.newBuilder().build()
+
+            ).build();
+
+        future = akka.pattern.Patterns.ask(subject, writeData, 3000);
+        assertTrue(future.isCompleted());
+        Await.result(future, Duration.Zero());
+
+
+    }
+
+    @Test(expected = IllegalStateException.class)
+    public void testNegativeMergeTransactionReady() throws Exception {
+
+
+        final ActorRef shard =
+            getSystem().actorOf(Shard.props("config", Collections.EMPTY_MAP));
+        final Props props =
+            ShardTransaction.props(store.newReadWriteTransaction(), shard,
+                TestModel.createTestContext());
+
+        final TestActorRef<ShardTransaction> subject = TestActorRef
+            .create(getSystem(), props, "testNegativeMergeTransactionReady");
+
+        ShardTransactionMessages.ReadyTransaction readyTransaction =
+            ShardTransactionMessages.ReadyTransaction.newBuilder().build();
+
+        Future<Object> future =
+            akka.pattern.Patterns.ask(subject, readyTransaction, 3000);
+        assertTrue(future.isCompleted());
+        Await.result(future, Duration.Zero());
+
+        ShardTransactionMessages.MergeData mergeData =
+            ShardTransactionMessages.MergeData.newBuilder()
+                .setInstanceIdentifierPathArguments(
+                    NormalizedNodeMessages.InstanceIdentifier.newBuilder()
+                        .build()).setNormalizedNode(
+                NormalizedNodeMessages.Node.newBuilder().build()
+
+            ).build();
+
+        future = akka.pattern.Patterns.ask(subject, mergeData, 3000);
+        assertTrue(future.isCompleted());
+        Await.result(future, Duration.Zero());
+
+
+    }
+
+
+    @Test(expected = IllegalStateException.class)
+    public void testNegativeDeleteDataWhenTransactionReady() throws Exception {
+
+
+        final ActorRef shard =
+            getSystem().actorOf(Shard.props("config", Collections.EMPTY_MAP));
+        final Props props =
+            ShardTransaction.props(store.newReadWriteTransaction(), shard,
+                TestModel.createTestContext());
+
+        final TestActorRef<ShardTransaction> subject = TestActorRef
+            .create(getSystem(), props,
+                "testNegativeDeleteDataWhenTransactionReady");
+
+        ShardTransactionMessages.ReadyTransaction readyTransaction =
+            ShardTransactionMessages.ReadyTransaction.newBuilder().build();
+
+        Future<Object> future =
+            akka.pattern.Patterns.ask(subject, readyTransaction, 3000);
+        assertTrue(future.isCompleted());
+        Await.result(future, Duration.Zero());
+
+        ShardTransactionMessages.DeleteData deleteData =
+            ShardTransactionMessages.DeleteData.newBuilder()
+                .setInstanceIdentifierPathArguments(
+                    NormalizedNodeMessages.InstanceIdentifier.newBuilder()
+                        .build()).build();
+
+        future = akka.pattern.Patterns.ask(subject, deleteData, 3000);
+        assertTrue(future.isCompleted());
+        Await.result(future, Duration.Zero());
+
+
+    }
+}
index f7c467652d329550dc665e53a2dd7b93e9342d12..41adcc55b188296f899b1569e98276906b233c1e 100644 (file)
@@ -8,48 +8,86 @@ import org.opendaylight.controller.cluster.datastore.jmx.mbeans.AbstractBaseMBea
 
 import javax.management.MBeanServer;
 import javax.management.ObjectName;
+import java.text.SimpleDateFormat;
+import java.util.Date;
 
 public class ShardStatsTest {
-  private MBeanServer mbeanServer;
private  ShardStats  shardStats;
-  private ObjectName testMBeanName;
+    private MBeanServer mbeanServer;
   private ShardStats shardStats;
+    private ObjectName testMBeanName;
 
-  @Before
-  public void setUp() throws Exception {
+    @Before
+    public void setUp() throws Exception {
 
-    shardStats = new ShardStats("shard-1");
-    shardStats.registerMBean();
-    mbeanServer= shardStats.getMBeanServer();
-    String objectName = AbstractBaseMBean.BASE_JMX_PREFIX + "type="+shardStats.getMBeanType()+",Category="+
-        shardStats.getMBeanCategory() + ",name="+
-        shardStats.getMBeanName();
-    testMBeanName = new ObjectName(objectName);
-  }
+        shardStats = new ShardStats("shard-1");
+        shardStats.registerMBean();
+        mbeanServer = shardStats.getMBeanServer();
+        String objectName =
+            AbstractBaseMBean.BASE_JMX_PREFIX + "type=" + shardStats
+                .getMBeanType() + ",Category=" +
+                shardStats.getMBeanCategory() + ",name=" +
+                shardStats.getMBeanName();
+        testMBeanName = new ObjectName(objectName);
+    }
 
-  @After
-  public void tearDown() throws Exception {
-    shardStats.unregisterMBean();
-  }
+    @After
+    public void tearDown() throws Exception {
+        shardStats.unregisterMBean();
+    }
 
-  @Test
-  public void testGetShardName() throws Exception {
+    @Test
+    public void testGetShardName() throws Exception {
 
-    Object attribute = mbeanServer.getAttribute(testMBeanName,"ShardName");
-    Assert.assertEquals((String) attribute, "shard-1");
+        Object attribute = mbeanServer.getAttribute(testMBeanName, "ShardName");
+        Assert.assertEquals((String) attribute, "shard-1");
 
-  }
+    }
 
-  @Test
-  public void testGetCommittedTransactionsCount() throws Exception {
-    //let us increment some transactions count and then check
-    shardStats.incrementCommittedTransactionCount();
-    shardStats.incrementCommittedTransactionCount();
-    shardStats.incrementCommittedTransactionCount();
+    @Test
+    public void testGetCommittedTransactionsCount() throws Exception {
+        //let us increment some transactions count and then check
+        shardStats.incrementCommittedTransactionCount();
+        shardStats.incrementCommittedTransactionCount();
+        shardStats.incrementCommittedTransactionCount();
 
-    //now let us get from MBeanServer what is the transaction count.
-    Object attribute = mbeanServer.getAttribute(testMBeanName,"CommittedTransactionsCount");
-    Assert.assertEquals((Long) attribute, (Long)3L);
+        //now let us get from MBeanServer what is the transaction count.
+        Object attribute = mbeanServer.getAttribute(testMBeanName,
+            "CommittedTransactionsCount");
+        Assert.assertEquals((Long) attribute, (Long) 3L);
 
 
-  }
-}
\ No newline at end of file
+    }
+
+    @Test
+    public void testGetLastCommittedTransactionTime() throws Exception {
+        SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS");
+        Assert.assertEquals(shardStats.getLastCommittedTransactionTime(),
+            sdf.format(new Date(0L)));
+        long millis = System.currentTimeMillis();
+        shardStats.setLastCommittedTransactionTime(new Date(millis));
+
+        //now let us get from MBeanServer what is the transaction count.
+        Object attribute = mbeanServer.getAttribute(testMBeanName,
+            "LastCommittedTransactionTime");
+        Assert.assertEquals((String) attribute, sdf.format(new Date(millis)));
+        Assert.assertNotEquals((String) attribute,
+            sdf.format(new Date(millis - 1)));
+
+    }
+
+    @Test
+    public void testGetFailedTransactionsCount() throws Exception {
+        //let us increment some transactions count and then check
+        shardStats.incrementFailedTransactionsCount();
+        shardStats.incrementFailedTransactionsCount();
+
+
+        //now let us get from MBeanServer what is the transaction count.
+        Object attribute =
+            mbeanServer.getAttribute(testMBeanName, "FailedTransactionsCount");
+        Assert.assertEquals((Long) attribute, (Long) 2L);
+
+
+
+    }
+}
index 2fdc8c7d1ef2e0873db819c2cb1b68cf88fb41d9..c1c57617082b0f79e6a3b9435eea2fd455ebad04 100644 (file)
@@ -35,4 +35,7 @@ and is available at http://www.eclipse.org/legal/epl-v10.html
       </services>
     </data>
   </configuration>
+  <required-capabilities>
+      <capability>urn:opendaylight:params:xml:ns:yang:controller:md:sal:rest:connector?module=opendaylight-rest-connector&amp;revision=2014-07-24</capability>
+  </required-capabilities>
 </snapshot>