X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?p=controller.git;a=blobdiff_plain;f=opendaylight%2Fmd-sal%2Fsal-distributed-datastore%2Fsrc%2Fmain%2Fjava%2Forg%2Fopendaylight%2Fcontroller%2Fcluster%2Fdatastore%2FShard.java;h=0fa27706e19382c0b84cc44b93d890ee9f0d1c8e;hp=b40ccd08e8fe6fbeae52845604a2bb032dd4cc15;hb=70c27e8bf6d323376a78aa5468faf4f27d081638;hpb=be4e53698d37883b68b96f45d5a71683ca7fd8e6 diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/Shard.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/Shard.java index b40ccd08e8..0fa27706e1 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/Shard.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/Shard.java @@ -26,14 +26,15 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.protobuf.ByteString; import com.google.protobuf.InvalidProtocolBufferException; +import org.opendaylight.controller.cluster.common.actor.CommonConfig; +import org.opendaylight.controller.cluster.common.actor.MeteringBehavior; import org.opendaylight.controller.cluster.datastore.identifiers.ShardIdentifier; import org.opendaylight.controller.cluster.datastore.identifiers.ShardTransactionIdentifier; import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shard.ShardMBeanFactory; import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shard.ShardStats; +import org.opendaylight.controller.cluster.datastore.messages.CloseTransactionChain; import org.opendaylight.controller.cluster.datastore.messages.CommitTransactionReply; import org.opendaylight.controller.cluster.datastore.messages.CreateTransaction; -import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionChain; -import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionChainReply; import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionReply; import org.opendaylight.controller.cluster.datastore.messages.EnableNotification; import org.opendaylight.controller.cluster.datastore.messages.ForwardedCommitTransaction; @@ -51,6 +52,7 @@ import org.opendaylight.controller.cluster.raft.DefaultConfigParamsImpl; import org.opendaylight.controller.cluster.raft.RaftActor; import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry; import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshotReply; +import org.opendaylight.controller.cluster.raft.protobuff.client.messages.CompositeModificationPayload; import org.opendaylight.controller.md.sal.common.api.data.AsyncDataChangeListener; import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException; import org.opendaylight.controller.md.sal.dom.store.impl.InMemoryDOMDataStore; @@ -59,6 +61,7 @@ import org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessa import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadTransaction; import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCohort; import org.opendaylight.controller.sal.core.spi.data.DOMStoreTransactionChain; +import org.opendaylight.controller.sal.core.spi.data.DOMStoreTransactionFactory; import org.opendaylight.controller.sal.core.spi.data.DOMStoreWriteTransaction; import org.opendaylight.yangtools.concepts.ListenerRegistration; import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier; @@ -67,7 +70,6 @@ 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; @@ -108,11 +110,12 @@ public class Shard extends RaftActor { private final DatastoreContext datastoreContext; - private SchemaContext schemaContext; private ActorRef createSnapshotTransaction; + private final Map transactionChains = new HashMap<>(); + private Shard(ShardIdentifier name, Map peerAddresses, DatastoreContext datastoreContext, SchemaContext schemaContext) { super(name.toString(), mapPeerAddresses(peerAddresses), Optional.of(configParams)); @@ -134,9 +137,14 @@ public class Shard extends RaftActor { store.onGlobalContextUpdated(schemaContext); } - shardMBean = ShardMBeanFactory.getShardStatsMBean(name.toString()); - + shardMBean = ShardMBeanFactory.getShardStatsMBean(name.toString(), + datastoreContext.getDataStoreMXBeanType()); + shardMBean.setDataStoreExecutor(store.getDomStoreExecutor()); + shardMBean.setNotificationManager(store.getDataChangeListenerNotificationManager()); + if (isMetricsCaptureEnabled()) { + getContext().become(new MeteringBehavior(this)); + } } private static Map mapPeerAddresses( @@ -163,8 +171,11 @@ public class Shard extends RaftActor { } @Override public void onReceiveRecover(Object message) { - LOG.debug("onReceiveRecover: Received message {} from {}", message.getClass().toString(), - getSender()); + if(LOG.isDebugEnabled()) { + LOG.debug("onReceiveRecover: Received message {} from {}", + message.getClass().toString(), + getSender()); + } if (message instanceof RecoveryFailure){ LOG.error(((RecoveryFailure) message).cause(), "Recovery failed because of this cause"); @@ -174,25 +185,26 @@ public class Shard extends RaftActor { } @Override public void onReceiveCommand(Object message) { - LOG.debug("onReceiveCommand: Received message {} from {}", message.getClass().toString(), - getSender()); + if(LOG.isDebugEnabled()) { + LOG.debug("onReceiveCommand: Received message {} from {}", + message.getClass().toString(), + getSender()); + } - if (message.getClass() - .equals(CreateTransactionChain.SERIALIZABLE_CLASS)) { - if (isLeader()) { - createTransactionChain(); - } else if (getLeader() != null) { - getLeader().forward(message, getContext()); - } - } else if(message.getClass().equals(ReadDataReply.SERIALIZABLE_CLASS)) { + if(message.getClass().equals(ReadDataReply.SERIALIZABLE_CLASS)) { // This must be for install snapshot. Don't want to open this up and trigger // deSerialization - self().tell(new CaptureSnapshotReply(ReadDataReply.getNormalizedNodeByteString(message)), self()); + self() + .tell(new CaptureSnapshotReply(ReadDataReply.getNormalizedNodeByteString(message)), + self()); + createSnapshotTransaction = null; // Send a PoisonPill instead of sending close transaction because we do not really need // a response getSender().tell(PoisonPill.getInstance(), self()); + } else if (message.getClass().equals(CloseTransactionChain.SERIALIZABLE_CLASS)){ + closeTransactionChain(CloseTransactionChain.fromSerializable(message)); } else if (message instanceof RegisterChangeListener) { registerChangeListener((RegisterChangeListener) message); } else if (message instanceof UpdateSchemaContext) { @@ -205,6 +217,9 @@ public class Shard extends RaftActor { createTransaction(CreateTransaction.fromSerializable(message)); } else if (getLeader() != null) { getLeader().forward(message, getContext()); + } else { + getSender().tell(new akka.actor.Status.Failure(new IllegalStateException( + "Could not find leader so transaction cannot be created")), getSelf()); } } else if (message instanceof PeerAddressResolved) { PeerAddressResolved resolved = (PeerAddressResolved) message; @@ -215,9 +230,30 @@ public class Shard extends RaftActor { } } + private void closeTransactionChain(CloseTransactionChain closeTransactionChain) { + DOMStoreTransactionChain chain = + transactionChains.remove(closeTransactionChain.getTransactionChainId()); + + if(chain != null) { + chain.close(); + } + } + private ActorRef createTypedTransactionActor( int transactionType, - ShardTransactionIdentifier transactionId) { + ShardTransactionIdentifier transactionId, + String transactionChainId ) { + + DOMStoreTransactionFactory factory = store; + + if(!transactionChainId.isEmpty()) { + factory = transactionChains.get(transactionChainId); + if(factory == null){ + DOMStoreTransactionChain transactionChain = store.createTransactionChain(); + transactionChains.put(transactionChainId, transactionChain); + factory = transactionChain; + } + } if(this.schemaContext == null){ throw new NullPointerException("schemaContext should not be null"); @@ -229,8 +265,8 @@ public class Shard extends RaftActor { shardMBean.incrementReadOnlyTransactionCount(); return getContext().actorOf( - ShardTransaction.props(store.newReadOnlyTransaction(), getSelf(), - schemaContext,datastoreContext, name.toString()), transactionId.toString()); + ShardTransaction.props(factory.newReadOnlyTransaction(), getSelf(), + schemaContext,datastoreContext, shardMBean), transactionId.toString()); } else if (transactionType == TransactionProxy.TransactionType.READ_WRITE.ordinal()) { @@ -238,8 +274,8 @@ public class Shard extends RaftActor { shardMBean.incrementReadWriteTransactionCount(); return getContext().actorOf( - ShardTransaction.props(store.newReadWriteTransaction(), getSelf(), - schemaContext, datastoreContext,name.toString()), transactionId.toString()); + ShardTransaction.props(factory.newReadWriteTransaction(), getSelf(), + schemaContext, datastoreContext, shardMBean), transactionId.toString()); } else if (transactionType @@ -248,8 +284,8 @@ public class Shard extends RaftActor { shardMBean.incrementWriteOnlyTransactionCount(); return getContext().actorOf( - ShardTransaction.props(store.newWriteOnlyTransaction(), getSelf(), - schemaContext, datastoreContext, name.toString()), transactionId.toString()); + ShardTransaction.props(factory.newWriteOnlyTransaction(), getSelf(), + schemaContext, datastoreContext, shardMBean), transactionId.toString()); } else { throw new IllegalArgumentException( "Shard="+name + ":CreateTransaction message has unidentified transaction type=" @@ -259,18 +295,20 @@ public class Shard extends RaftActor { private void createTransaction(CreateTransaction createTransaction) { createTransaction(createTransaction.getTransactionType(), - createTransaction.getTransactionId()); + createTransaction.getTransactionId(), createTransaction.getTransactionChainId()); } - private ActorRef createTransaction(int transactionType, String remoteTransactionId) { + private ActorRef createTransaction(int transactionType, String remoteTransactionId, String transactionChainId) { ShardTransactionIdentifier transactionId = ShardTransactionIdentifier.builder() .remoteTransactionId(remoteTransactionId) .build(); - LOG.debug("Creating transaction : {} ", transactionId); + if(LOG.isDebugEnabled()) { + LOG.debug("Creating transaction : {} ", transactionId); + } ActorRef transactionActor = - createTypedTransactionActor(transactionType, transactionId); + createTypedTransactionActor(transactionType, transactionId, transactionChainId); getSender() .tell(new CreateTransactionReply( @@ -295,11 +333,20 @@ public class Shard extends RaftActor { DOMStoreThreePhaseCommitCohort cohort = modificationToCohort.remove(serialized); if (cohort == null) { - LOG.debug( - "Could not find cohort for modification : {}. Writing modification using a new transaction", - modification); + + if(LOG.isDebugEnabled()) { + LOG.debug( + "Could not find cohort for modification : {}. Writing modification using a new transaction", + modification); + } + DOMStoreWriteTransaction transaction = store.newWriteOnlyTransaction(); + + if(LOG.isDebugEnabled()) { + LOG.debug("Created new transaction {}", transaction.getIdentifier().toString()); + } + modification.apply(transaction); try { syncCommitTransaction(transaction); @@ -313,22 +360,27 @@ public class Shard extends RaftActor { return; } - final ListenableFuture future = cohort.commit(); - final ActorRef self = getSelf(); + + if(sender == null){ + LOG.error("Commit failed. Sender cannot be null"); + return; + } + + ListenableFuture future = cohort.commit(); Futures.addCallback(future, new FutureCallback() { @Override public void onSuccess(Void v) { - sender.tell(new CommitTransactionReply().toSerializable(), self); + sender.tell(new CommitTransactionReply().toSerializable(), getSelf()); shardMBean.incrementCommittedTransactionCount(); - shardMBean.setLastCommittedTransactionTime(new Date()); + shardMBean.setLastCommittedTransactionTime(System.currentTimeMillis()); } @Override public void onFailure(Throwable t) { LOG.error(t, "An exception happened during commit"); shardMBean.incrementFailedTransactionsCount(); - sender.tell(new akka.actor.Status.Failure(t), self); + sender.tell(new akka.actor.Status.Failure(t), getSelf()); } }); @@ -362,8 +414,10 @@ public class Shard extends RaftActor { private void registerChangeListener( RegisterChangeListener registerChangeListener) { - LOG.debug("registerDataChangeListener for {}", registerChangeListener - .getPath()); + if(LOG.isDebugEnabled()) { + LOG.debug("registerDataChangeListener for {}", registerChangeListener + .getPath()); + } ActorSelection dataChangeListenerPath = getContext() @@ -391,21 +445,20 @@ public class Shard extends RaftActor { getContext().actorOf( DataChangeListenerRegistration.props(registration)); - LOG.debug( - "registerDataChangeListener sending reply, listenerRegistrationPath = {} " - , listenerRegistration.path().toString()); + if(LOG.isDebugEnabled()) { + LOG.debug( + "registerDataChangeListener sending reply, listenerRegistrationPath = {} " + , listenerRegistration.path().toString()); + } getSender() .tell(new RegisterChangeListenerReply(listenerRegistration.path()), getSelf()); } - private void createTransactionChain() { - DOMStoreTransactionChain chain = store.createTransactionChain(); - ActorRef transactionChain = getContext().actorOf( - ShardTransactionChain.props(chain, schemaContext, datastoreContext,name.toString() )); - getSender().tell(new CreateTransactionChainReply(transactionChain.path()).toSerializable(), - getSelf()); + private boolean isMetricsCaptureEnabled(){ + CommonConfig config = new CommonConfig(getContext().system().settings().config()); + return config.isMetricCaptureEnabled(); } @Override protected void applyState(ActorRef clientActor, String identifier, @@ -424,7 +477,7 @@ public class Shard extends RaftActor { } } else { - LOG.error("Unknown state received {}", data); + LOG.error("Unknown state received {} Class loader = {} CompositeNodeMod.ClassLoader = {}", data, data.getClass().getClassLoader(), CompositeModificationPayload.class.getClassLoader()); } // Update stats @@ -447,7 +500,7 @@ public class Shard extends RaftActor { // so that this actor does not get block building the snapshot createSnapshotTransaction = createTransaction( TransactionProxy.TransactionType.READ_ONLY.ordinal(), - "createSnapshot"); + "createSnapshot", ""); createSnapshotTransaction.tell( new ReadData(YangInstanceIdentifier.builder().build()).toSerializable(), self()); @@ -459,6 +512,8 @@ public class Shard extends RaftActor { // Since this will be done only on Recovery or when this actor is a Follower // we can safely commit everything in here. We not need to worry about event notifications // as they would have already been disabled on the follower + + LOG.info("Applying snapshot"); try { DOMStoreWriteTransaction transaction = store.newWriteOnlyTransaction(); NormalizedNodeMessages.Node serializedNode = NormalizedNodeMessages.Node.parseFrom(snapshot); @@ -473,6 +528,8 @@ public class Shard extends RaftActor { syncCommitTransaction(transaction); } catch (InvalidProtocolBufferException | InterruptedException | ExecutionException e) { LOG.error(e, "An exception occurred when applying snapshot"); + } finally { + LOG.info("Done applying snapshot"); } } @@ -482,12 +539,26 @@ public class Shard extends RaftActor { .tell(new EnableNotification(isLeader()), getSelf()); } - if (getLeaderId() != null) { - shardMBean.setLeader(getLeaderId()); - } - shardMBean.setRaftState(getRaftState().name()); shardMBean.setCurrentTerm(getCurrentTerm()); + + // If this actor is no longer the leader close all the transaction chains + if(!isLeader()){ + for(Map.Entry entry : transactionChains.entrySet()){ + if(LOG.isDebugEnabled()) { + LOG.debug( + "onStateChanged: Closing transaction chain {} because shard {} is no longer the leader", + entry.getKey(), getId()); + } + entry.getValue().close(); + } + + transactionChains.clear(); + } + } + + @Override protected void onLeaderChanged(String oldLeader, String newLeader) { + shardMBean.setLeader(newLeader); } @Override public String persistenceId() {