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=5b8a50e18896c0a3e10db64c10b1d6fdffdc41d8;hp=dabad41aff97b81ba83dcdf1626dca26e987a1b1;hb=a51163ead1d60e66eeaf3691adb70b019ce60fb2;hpb=9402730d96f9633d2165de99fad65768d8537a9a 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 dabad41aff..5b8a50e188 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 @@ -12,14 +12,13 @@ import akka.actor.ActorRef; import akka.actor.ActorSelection; import akka.actor.Cancellable; import akka.actor.Props; -import akka.japi.Creator; import akka.persistence.RecoveryFailure; import akka.serialization.Serialization; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Optional; import com.google.common.base.Preconditions; import java.io.IOException; -import java.util.HashMap; +import java.util.Collections; import java.util.Map; import java.util.concurrent.TimeUnit; import javax.annotation.Nonnull; @@ -40,6 +39,8 @@ import org.opendaylight.controller.cluster.datastore.messages.CommitTransaction; import org.opendaylight.controller.cluster.datastore.messages.CommitTransactionReply; import org.opendaylight.controller.cluster.datastore.messages.CreateTransaction; import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionReply; +import org.opendaylight.controller.cluster.datastore.messages.DatastoreSnapshot; +import org.opendaylight.controller.cluster.datastore.messages.DatastoreSnapshot.ShardSnapshot; import org.opendaylight.controller.cluster.datastore.messages.ForwardedReadyTransaction; import org.opendaylight.controller.cluster.datastore.messages.GetShardDataTree; import org.opendaylight.controller.cluster.datastore.messages.PeerAddressResolved; @@ -62,12 +63,14 @@ import org.opendaylight.controller.cluster.raft.RaftActorSnapshotCohort; import org.opendaylight.controller.cluster.raft.RaftState; import org.opendaylight.controller.cluster.raft.base.messages.FollowerInitialSyncUpStatus; import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply; +import org.opendaylight.controller.cluster.raft.messages.ServerRemoved; import org.opendaylight.controller.cluster.raft.protobuff.client.messages.CompositeModificationByteStringPayload; import org.opendaylight.controller.cluster.raft.protobuff.client.messages.CompositeModificationPayload; 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.DataValidationFailedException; import org.opendaylight.yangtools.yang.data.api.schema.tree.ModificationType; +import org.opendaylight.yangtools.yang.data.api.schema.tree.TreeType; import org.opendaylight.yangtools.yang.model.api.SchemaContext; import scala.concurrent.duration.Duration; import scala.concurrent.duration.FiniteDuration; @@ -115,19 +118,24 @@ public class Shard extends RaftActor { private final DataTreeChangeListenerSupport treeChangeSupport = new DataTreeChangeListenerSupport(this); private final DataChangeListenerSupport changeSupport = new DataChangeListenerSupport(this); - protected Shard(final ShardIdentifier name, final Map peerAddresses, - final DatastoreContext datastoreContext, final SchemaContext schemaContext) { - super(name.toString(), new HashMap<>(peerAddresses), Optional.of(datastoreContext.getShardRaftConfig()), - DataStoreVersions.CURRENT_VERSION); - this.name = name.toString(); - this.datastoreContext = datastoreContext; + private ShardSnapshot restoreFromSnapshot; + + private final ShardTransactionMessageRetrySupport messageRetrySupport; + + protected Shard(AbstractBuilder builder) { + super(builder.getId().toString(), builder.getPeerAddresses(), + Optional.of(builder.getDatastoreContext().getShardRaftConfig()), DataStoreVersions.CURRENT_VERSION); + + this.name = builder.getId().toString(); + this.datastoreContext = builder.getDatastoreContext(); + this.restoreFromSnapshot = builder.getRestoreFromSnapshot(); setPersistence(datastoreContext.isPersistent()); LOG.info("Shard created : {}, persistent : {}", name, datastoreContext.isPersistent()); - store = new ShardDataTree(schemaContext); + store = new ShardDataTree(builder.getSchemaContext(), builder.getTreeType()); shardMBean = ShardMBeanFactory.getShardStatsMBean(name.toString(), datastoreContext.getDataStoreMXBeanType()); @@ -139,7 +147,7 @@ public class Shard extends RaftActor { commitCoordinator = new ShardCommitCoordinator(store, datastoreContext.getShardCommitQueueExpiryTimeoutInMillis(), - datastoreContext.getShardTransactionCommitQueueCapacity(), self(), LOG, this.name); + datastoreContext.getShardTransactionCommitQueueCapacity(), LOG, this.name); setTransactionCommitTimeout(); @@ -155,7 +163,7 @@ public class Shard extends RaftActor { snapshotCohort = new ShardSnapshotCohort(transactionActorFactory, store, LOG, this.name); - + messageRetrySupport = new ShardTransactionMessageRetrySupport(this); } private void setTransactionCommitTimeout() { @@ -163,11 +171,6 @@ public class Shard extends RaftActor { datastoreContext.getShardTransactionCommitTimeoutInSeconds(), TimeUnit.SECONDS) / 2; } - public static Props props(final ShardIdentifier name, final Map peerAddresses, - final DatastoreContext datastoreContext, final SchemaContext schemaContext) { - return Props.create(new ShardCreator(name, peerAddresses, datastoreContext, schemaContext)); - } - private Optional createRoleChangeNotifier(String shardId) { ActorRef shardRoleChangeNotifier = this.getContext().actorOf( RoleChangeNotifier.getProps(shardId), shardId + "-notifier"); @@ -180,6 +183,8 @@ public class Shard extends RaftActor { super.postStop(); + messageRetrySupport.close(); + if(txCommitTimeoutCheckSchedule != null) { txCommitTimeoutCheckSchedule.cancel(); } @@ -225,8 +230,7 @@ public class Shard extends RaftActor { } else if (BatchedModifications.class.isInstance(message)) { handleBatchedModifications((BatchedModifications)message); } else if (message instanceof ForwardedReadyTransaction) { - commitCoordinator.handleForwardedReadyTransaction((ForwardedReadyTransaction) message, - getSender(), this); + handleForwardedReadyTransaction((ForwardedReadyTransaction) message); } else if (message instanceof ReadyLocalTransaction) { handleReadyLocalTransaction((ReadyLocalTransaction)message); } else if (CanCommitTransaction.SERIALIZABLE_CLASS.isInstance(message)) { @@ -258,8 +262,12 @@ public class Shard extends RaftActor { context().parent().tell(message, self()); } else if(GET_SHARD_MBEAN_MESSAGE.equals(message)){ sender().tell(getShardMBean(), self()); - } else if(message instanceof GetShardDataTree){ + } else if(message instanceof GetShardDataTree) { sender().tell(store.getDataTree(), self()); + } else if(message instanceof ServerRemoved){ + context().parent().forward(message, context()); + } else if(ShardTransactionMessageRetrySupport.TIMER_MESSAGE_CLASS.isInstance(message)) { + messageRetrySupport.onTimerMessage(message); } else { super.onReceiveCommand(message); } @@ -332,7 +340,7 @@ public class Shard extends RaftActor { applyModificationToState(cohortEntry.getReplySender(), cohortEntry.getTransactionID(), candidate); } else { Shard.this.persistData(cohortEntry.getReplySender(), cohortEntry.getTransactionID(), - DataTreeCandidatePayload.create(candidate)); + DataTreeCandidatePayload.create(candidate)); } } @@ -405,12 +413,6 @@ public class Shard extends RaftActor { commitCoordinator.handleCanCommit(canCommit.getTransactionID(), getSender(), this); } - private void noLeaderError(String errMessage, Object message) { - // TODO: rather than throwing an immediate exception, we could schedule a timer to try again to make - // it more resilient in case we're in the process of electing a new leader. - getSender().tell(new akka.actor.Status.Failure(new NoShardLeaderException(errMessage, persistenceId())), getSelf()); - } - protected void handleBatchedModificationsLocal(BatchedModifications batched, ActorRef sender) { try { commitCoordinator.handleBatchedModifications(batched, sender, this); @@ -447,7 +449,8 @@ public class Shard extends RaftActor { LOG.debug("{}: Forwarding BatchedModifications to leader {}", persistenceId(), leader); leader.forward(batched, getContext()); } else { - noLeaderError("Could not commit transaction " + batched.getTransactionID(), batched); + messageRetrySupport.addMessageToRetry(batched, getSender(), + "Could not commit transaction " + batched.getTransactionID()); } } } @@ -486,7 +489,29 @@ public class Shard extends RaftActor { message.setRemoteVersion(getCurrentBehavior().getLeaderPayloadVersion()); leader.forward(message, getContext()); } else { - noLeaderError("Could not commit transaction " + message.getTransactionID(), message); + messageRetrySupport.addMessageToRetry(message, getSender(), + "Could not commit transaction " + message.getTransactionID()); + } + } + } + + private void handleForwardedReadyTransaction(ForwardedReadyTransaction forwardedReady) { + if (isLeader()) { + failIfIsolatedLeader(getSender()); + + commitCoordinator.handleForwardedReadyTransaction(forwardedReady, getSender(), this); + } else { + ActorSelection leader = getLeader(); + if (leader != null) { + LOG.debug("{}: Forwarding ForwardedReadyTransaction to leader {}", persistenceId(), leader); + + ReadyLocalTransaction readyLocal = new ReadyLocalTransaction(forwardedReady.getTransactionID(), + forwardedReady.getTransaction().getSnapshot(), forwardedReady.isDoImmediateCommit()); + readyLocal.setRemoteVersion(getCurrentBehavior().getLeaderPayloadVersion()); + leader.forward(readyLocal, getContext()); + } else { + messageRetrySupport.addMessageToRetry(forwardedReady, getSender(), + "Could not commit transaction " + forwardedReady.getTransactionID()); } } } @@ -592,11 +617,14 @@ public class Shard extends RaftActor { @Override @Nonnull protected RaftActorRecoveryCohort getRaftActorRecoveryCohort() { - return new ShardRecoveryCoordinator(store, store.getSchemaContext(), persistenceId(), LOG); + return new ShardRecoveryCoordinator(store, store.getSchemaContext(), + restoreFromSnapshot != null ? restoreFromSnapshot.getSnapshot() : null, persistenceId(), LOG); } @Override protected void onRecoveryComplete() { + restoreFromSnapshot = null; + //notify shard manager getContext().parent().tell(new ActorInitialized(), getSelf()); @@ -683,6 +711,10 @@ public class Shard extends RaftActor { @Override protected void onLeaderChanged(String oldLeader, String newLeader) { shardMBean.incrementLeadershipChangeCount(); + + if(hasLeader()) { + messageRetrySupport.retryMessages(); + } } @Override @@ -699,44 +731,120 @@ public class Shard extends RaftActor { return datastoreContext; } - protected abstract static class AbstractShardCreator implements Creator { - private static final long serialVersionUID = 1L; + @VisibleForTesting + public ShardDataTree getDataStore() { + return store; + } - protected final ShardIdentifier name; - protected final Map peerAddresses; - protected final DatastoreContext datastoreContext; - protected final SchemaContext schemaContext; + @VisibleForTesting + ShardStats getShardMBean() { + return shardMBean; + } - protected AbstractShardCreator(final ShardIdentifier name, final Map peerAddresses, - final DatastoreContext datastoreContext, final SchemaContext schemaContext) { - this.name = Preconditions.checkNotNull(name, "name should not be null"); - this.peerAddresses = Preconditions.checkNotNull(peerAddresses, "peerAddresses should not be null"); - this.datastoreContext = Preconditions.checkNotNull(datastoreContext, "dataStoreContext should not be null"); - this.schemaContext = Preconditions.checkNotNull(schemaContext, "schemaContext should not be null"); - } + public static Builder builder() { + return new Builder(); } - private static class ShardCreator extends AbstractShardCreator { - private static final long serialVersionUID = 1L; + public static abstract class AbstractBuilder, S extends Shard> { + private final Class shardClass; + private ShardIdentifier id; + private Map peerAddresses = Collections.emptyMap(); + private DatastoreContext datastoreContext; + private SchemaContext schemaContext; + private DatastoreSnapshot.ShardSnapshot restoreFromSnapshot; + private volatile boolean sealed; - ShardCreator(final ShardIdentifier name, final Map peerAddresses, - final DatastoreContext datastoreContext, final SchemaContext schemaContext) { - super(name, peerAddresses, datastoreContext, schemaContext); + protected AbstractBuilder(Class shardClass) { + this.shardClass = shardClass; } - @Override - public Shard create() throws Exception { - return new Shard(name, peerAddresses, datastoreContext, schemaContext); + protected void checkSealed() { + Preconditions.checkState(!sealed, "Builder isalready sealed - further modifications are not allowed"); } - } - @VisibleForTesting - public ShardDataTree getDataStore() { - return store; + @SuppressWarnings("unchecked") + private T self() { + return (T) this; + } + + public T id(ShardIdentifier id) { + checkSealed(); + this.id = id; + return self(); + } + + public T peerAddresses(Map peerAddresses) { + checkSealed(); + this.peerAddresses = peerAddresses; + return self(); + } + + public T datastoreContext(DatastoreContext datastoreContext) { + checkSealed(); + this.datastoreContext = datastoreContext; + return self(); + } + + public T schemaContext(SchemaContext schemaContext) { + checkSealed(); + this.schemaContext = schemaContext; + return self(); + } + + public T restoreFromSnapshot(DatastoreSnapshot.ShardSnapshot restoreFromSnapshot) { + checkSealed(); + this.restoreFromSnapshot = restoreFromSnapshot; + return self(); + } + + public ShardIdentifier getId() { + return id; + } + + public Map getPeerAddresses() { + return peerAddresses; + } + + public DatastoreContext getDatastoreContext() { + return datastoreContext; + } + + public SchemaContext getSchemaContext() { + return schemaContext; + } + + public DatastoreSnapshot.ShardSnapshot getRestoreFromSnapshot() { + return restoreFromSnapshot; + } + + public TreeType getTreeType() { + switch (datastoreContext.getLogicalStoreType()) { + case CONFIGURATION: + return TreeType.CONFIGURATION; + case OPERATIONAL: + return TreeType.OPERATIONAL; + } + + throw new IllegalStateException("Unhandled logical store type " + datastoreContext.getLogicalStoreType()); + } + + protected void verify() { + Preconditions.checkNotNull(id, "id should not be null"); + Preconditions.checkNotNull(peerAddresses, "peerAddresses should not be null"); + Preconditions.checkNotNull(datastoreContext, "dataStoreContext should not be null"); + Preconditions.checkNotNull(schemaContext, "schemaContext should not be null"); + } + + public Props props() { + sealed = true; + verify(); + return Props.create(shardClass, this); + } } - @VisibleForTesting - ShardStats getShardMBean() { - return shardMBean; + public static class Builder extends AbstractBuilder { + private Builder() { + super(Shard.class); + } } }