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=4440c8972db91fca202262ec6a0d8f0b0e5b7752;hp=26295fe852ef6dea3578c3f6697d29d8879d2866;hb=0f6310d201d4294b8c9e156ca1202e1bc52ce6eb;hpb=2f77e92af7a68b4a97dbfb709c6cc9b11a49878a 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 26295fe852..4440c8972d 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,19 +12,21 @@ import akka.actor.ActorRef; import akka.actor.ActorSelection; import akka.actor.Cancellable; import akka.actor.Props; +import akka.actor.Status; import akka.actor.Status.Failure; import akka.serialization.Serialization; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.common.base.Ticker; +import com.google.common.base.Verify; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Range; import java.io.IOException; import java.util.Arrays; import java.util.Collection; import java.util.Collections; -import java.util.HashMap; import java.util.Map; import java.util.concurrent.TimeUnit; import javax.annotation.Nonnull; @@ -37,6 +39,7 @@ import org.opendaylight.controller.cluster.access.commands.NotLeaderException; import org.opendaylight.controller.cluster.access.commands.TransactionRequest; import org.opendaylight.controller.cluster.access.concepts.ClientIdentifier; import org.opendaylight.controller.cluster.access.concepts.FrontendIdentifier; +import org.opendaylight.controller.cluster.access.concepts.LocalHistoryIdentifier; import org.opendaylight.controller.cluster.access.concepts.Request; import org.opendaylight.controller.cluster.access.concepts.RequestEnvelope; import org.opendaylight.controller.cluster.access.concepts.RequestException; @@ -63,19 +66,23 @@ import org.opendaylight.controller.cluster.datastore.messages.CreateTransaction; import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionReply; import org.opendaylight.controller.cluster.datastore.messages.ForwardedReadyTransaction; import org.opendaylight.controller.cluster.datastore.messages.GetShardDataTree; +import org.opendaylight.controller.cluster.datastore.messages.MakeLeaderLocal; import org.opendaylight.controller.cluster.datastore.messages.OnDemandShardState; import org.opendaylight.controller.cluster.datastore.messages.PeerAddressResolved; +import org.opendaylight.controller.cluster.datastore.messages.PersistAbortTransactionPayload; import org.opendaylight.controller.cluster.datastore.messages.ReadyLocalTransaction; import org.opendaylight.controller.cluster.datastore.messages.RegisterChangeListener; import org.opendaylight.controller.cluster.datastore.messages.RegisterDataTreeChangeListener; import org.opendaylight.controller.cluster.datastore.messages.ShardLeaderStateChanged; import org.opendaylight.controller.cluster.datastore.messages.UpdateSchemaContext; +import org.opendaylight.controller.cluster.datastore.persisted.AbortTransactionPayload; import org.opendaylight.controller.cluster.datastore.persisted.DatastoreSnapshot; import org.opendaylight.controller.cluster.datastore.persisted.DatastoreSnapshot.ShardSnapshot; import org.opendaylight.controller.cluster.datastore.utils.Dispatchers; import org.opendaylight.controller.cluster.notifications.LeaderStateChanged; import org.opendaylight.controller.cluster.notifications.RegisterRoleChangeListener; import org.opendaylight.controller.cluster.notifications.RoleChangeNotifier; +import org.opendaylight.controller.cluster.raft.LeadershipTransferFailedException; import org.opendaylight.controller.cluster.raft.RaftActor; import org.opendaylight.controller.cluster.raft.RaftActorRecoveryCohort; import org.opendaylight.controller.cluster.raft.RaftActorSnapshotCohort; @@ -83,6 +90,7 @@ import org.opendaylight.controller.cluster.raft.RaftState; import org.opendaylight.controller.cluster.raft.base.messages.FollowerInitialSyncUpStatus; import org.opendaylight.controller.cluster.raft.client.messages.OnDemandRaftState; import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply; +import org.opendaylight.controller.cluster.raft.messages.RequestLeadership; import org.opendaylight.controller.cluster.raft.messages.ServerRemoved; import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload; import org.opendaylight.yangtools.concepts.Identifier; @@ -90,6 +98,7 @@ import org.opendaylight.yangtools.yang.data.api.schema.tree.DataValidationFailed import org.opendaylight.yangtools.yang.data.api.schema.tree.TipProducingDataTree; import org.opendaylight.yangtools.yang.data.api.schema.tree.TreeType; import org.opendaylight.yangtools.yang.model.api.SchemaContext; +import org.opendaylight.yangtools.yang.model.api.SchemaContextProvider; import scala.concurrent.duration.Duration; import scala.concurrent.duration.FiniteDuration; @@ -117,6 +126,13 @@ public class Shard extends RaftActor { } }; + static final Object RESUME_NEXT_PENDING_TRANSACTION = new Object() { + @Override + public String toString() { + return "resumeNextPendingTransaction"; + } + }; + // FIXME: shard names should be encapsulated in their own class and this should be exposed as a constant. public static final String DEFAULT_NAME = "default"; @@ -163,8 +179,8 @@ public class Shard extends RaftActor { private final ShardTransactionMessageRetrySupport messageRetrySupport; - private final FrontendMetadata frontendMetadata = new FrontendMetadata(); - private final Map knownFrontends = new HashMap<>(); + private final FrontendMetadata frontendMetadata; + private Map knownFrontends = ImmutableMap.of(); protected Shard(final AbstractBuilder builder) { super(builder.getId().toString(), builder.getPeerAddresses(), @@ -173,22 +189,23 @@ public class Shard extends RaftActor { this.name = builder.getId().toString(); this.datastoreContext = builder.getDatastoreContext(); this.restoreFromSnapshot = builder.getRestoreFromSnapshot(); + this.frontendMetadata = new FrontendMetadata(name); setPersistence(datastoreContext.isPersistent()); LOG.info("Shard created : {}, persistent : {}", name, datastoreContext.isPersistent()); ShardDataTreeChangeListenerPublisherActorProxy treeChangeListenerPublisher = - new ShardDataTreeChangeListenerPublisherActorProxy(getContext(), name + "-DTCL-publisher"); + new ShardDataTreeChangeListenerPublisherActorProxy(getContext(), name + "-DTCL-publisher", name); ShardDataChangeListenerPublisherActorProxy dataChangeListenerPublisher = - new ShardDataChangeListenerPublisherActorProxy(getContext(), name + "-DCL-publisher"); + new ShardDataChangeListenerPublisherActorProxy(getContext(), name + "-DCL-publisher", name); if (builder.getDataTree() != null) { store = new ShardDataTree(this, builder.getSchemaContext(), builder.getDataTree(), - treeChangeListenerPublisher, dataChangeListenerPublisher, name); + treeChangeListenerPublisher, dataChangeListenerPublisher, name, frontendMetadata); } else { store = new ShardDataTree(this, builder.getSchemaContext(), builder.getTreeType(), builder.getDatastoreContext().getStoreRoot(), treeChangeListenerPublisher, - dataChangeListenerPublisher, name); + dataChangeListenerPublisher, name, frontendMetadata); } shardMBean = ShardMBeanFactory.getShardStatsMBean(name, datastoreContext.getDataStoreMXBeanType(), this); @@ -208,8 +225,8 @@ public class Shard extends RaftActor { getRaftActorContext().getConfigParams().getIsolatedCheckIntervalInMillis()); transactionActorFactory = new ShardTransactionActorFactory(store, datastoreContext, - new Dispatchers(context().system().dispatchers()).getDispatcherPath( - Dispatchers.DispatcherType.Transaction), self(), getContext(), shardMBean); + new Dispatchers(context().system().dispatchers()).getDispatcherPath(Dispatchers.DispatcherType.Transaction), + self(), getContext(), shardMBean, builder.getId().getShardName()); snapshotCohort = ShardSnapshotCohort.create(getContext(), builder.getId().getMemberName(), store, LOG, this.name); @@ -266,6 +283,8 @@ public class Shard extends RaftActor { maybeError.get()); } + store.resetTransactionBatch(); + if (message instanceof RequestEnvelope) { final long now = ticker().read(); final RequestEnvelope envelope = (RequestEnvelope)message; @@ -331,12 +350,45 @@ public class Shard extends RaftActor { } else if (message instanceof DataTreeCohortActorRegistry.CohortRegistryCommand) { store.processCohortRegistryCommand(getSender(), (DataTreeCohortActorRegistry.CohortRegistryCommand) message); + } else if (message instanceof PersistAbortTransactionPayload) { + final TransactionIdentifier txId = ((PersistAbortTransactionPayload) message).getTransactionId(); + persistPayload(txId, AbortTransactionPayload.create(txId), true); + } else if (message instanceof MakeLeaderLocal) { + onMakeLeaderLocal(); + } else if (RESUME_NEXT_PENDING_TRANSACTION.equals(message)) { + store.resumeNextPendingTransaction(); } else { super.handleNonRaftCommand(message); } } } + private void onMakeLeaderLocal() { + LOG.debug("{}: onMakeLeaderLocal received", persistenceId()); + if (isLeader()) { + getSender().tell(new Status.Success(null), getSelf()); + return; + } + + final ActorSelection leader = getLeader(); + + if (leader == null) { + // Leader is not present. The cluster is most likely trying to + // elect a leader and we should let that run its normal course + + // TODO we can wait for the election to complete and retry the + // request. We can also let the caller retry by sending a flag + // in the response indicating the request is "reTryable". + getSender().tell(new Failure( + new LeadershipTransferFailedException("We cannot initiate leadership transfer to local node. " + + "Currently there is no leader for " + persistenceId())), + getSelf()); + return; + } + + leader.tell(new RequestLeadership(getId(), getSender()), getSelf()); + } + // Acquire our frontend tracking handle and verify generation matches private LeaderFrontendState getFrontend(final ClientIdentifier clientId) throws RequestException { final LeaderFrontendState existing = knownFrontends.get(clientId.getFrontendId()); @@ -380,7 +432,9 @@ public class Shard extends RaftActor { private void handleConnectClient(final ConnectClientRequest message) { try { if (!isLeader() || !isLeaderActive()) { - LOG.debug("{}: not currently leader, rejecting request {}", persistenceId(), message); + LOG.info("{}: not currently leader, rejecting request {}. isLeader: {}, isLeaderActive: {}," + + "isLeadershipTransferInProgress: {}.", + persistenceId(), message, isLeader(), isLeaderActive(), isLeadershipTransferInProgress()); throw new NotLeaderException(getSelf()); } @@ -399,7 +453,9 @@ public class Shard extends RaftActor { throws RequestException { // We are not the leader, hence we want to fail-fast. if (!isLeader() || !isLeaderActive()) { - LOG.debug("{}: not currently leader, rejecting request {}", persistenceId(), envelope); + LOG.info("{}: not currently leader, rejecting request {}. isLeader: {}, isLeaderActive: {}," + + "isLeadershipTransferInProgress: {}.", + persistenceId(), envelope, isLeader(), isLeaderActive(), isLeadershipTransferInProgress()); throw new NotLeaderException(getSelf()); } @@ -413,7 +469,7 @@ public class Shard extends RaftActor { final ClientIdentifier clientId = lhReq.getTarget().getClientId(); return getFrontend(clientId).handleLocalHistoryRequest(lhReq, envelope, now); } else { - LOG.debug("{}: rejecting unsupported request {}", persistenceId(), request); + LOG.warn("{}: rejecting unsupported request {}", persistenceId(), request); throw new UnsupportedRequestException(request); } } @@ -453,13 +509,13 @@ public class Shard extends RaftActor { } // applyState() will be invoked once consensus is reached on the payload - void persistPayload(final TransactionIdentifier transactionId, final Payload payload, boolean batchHint) { + void persistPayload(final Identifier id, final Payload payload, final boolean batchHint) { boolean canSkipPayload = !hasFollowers() && !persistence().isRecoveryApplicable(); if (canSkipPayload) { - applyState(self(), transactionId, payload); + applyState(self(), id, payload); } else { // We are faking the sender - persistData(self(), transactionId, payload, batchHint); + persistData(self(), id, payload, batchHint); } } @@ -614,7 +670,7 @@ public class Shard extends RaftActor { doAbortTransaction(abort.getTransactionId(), getSender()); } - void doAbortTransaction(final TransactionIdentifier transactionID, final ActorRef sender) { + void doAbortTransaction(final Identifier transactionID, final ActorRef sender) { commitCoordinator.handleAbort(transactionID, sender, this); } @@ -630,7 +686,9 @@ public class Shard extends RaftActor { } private void closeTransactionChain(final CloseTransactionChain closeTransactionChain) { - store.closeTransactionChain(closeTransactionChain.getIdentifier()); + final LocalHistoryIdentifier id = closeTransactionChain.getIdentifier(); + store.closeTransactionChain(id, null); + store.purgeTransactionChain(id, null); } @SuppressWarnings("checkstyle:IllegalCatch") @@ -730,7 +788,7 @@ public class Shard extends RaftActor { persistenceId(), getId()); } - store.closeAllTransactionChains(); + store.purgeLeaderState(); } if (hasLeader && !isIsolatedLeader()) { @@ -742,8 +800,18 @@ public class Shard extends RaftActor { protected void onLeaderChanged(final String oldLeader, final String newLeader) { shardMBean.incrementLeadershipChangeCount(); - boolean hasLeader = hasLeader(); - if (hasLeader && !isLeader()) { + final boolean hasLeader = hasLeader(); + if (!hasLeader) { + // No leader implies we are not the leader, lose frontend state if we have any. This also places + // an explicit guard so the map will not get modified accidentally. + if (!knownFrontends.isEmpty()) { + LOG.debug("{}: removing frontend state for {}", persistenceId(), knownFrontends.keySet()); + knownFrontends = ImmutableMap.of(); + } + return; + } + + if (!isLeader()) { // Another leader was elected. If we were the previous leader and had pending transactions, convert // them to transaction messages and send to the new leader. ActorSelection leader = getLeader(); @@ -762,9 +830,13 @@ public class Shard extends RaftActor { commitCoordinator.abortPendingTransactions("The transacton was aborted due to inflight leadership " + "change and the leader address isn't available.", this); } + } else { + // We have become the leader, we need to reconstruct frontend state + knownFrontends = Verify.verifyNotNull(frontendMetadata.toLeaderState(this)); + LOG.debug("{}: became leader with frontend state for {}", persistenceId(), knownFrontends.keySet()); } - if (hasLeader && !isIsolatedLeader()) { + if (!isIsolatedLeader()) { messageRetrySupport.retryMessages(); } } @@ -825,7 +897,7 @@ public class Shard extends RaftActor { private ShardIdentifier id; private Map peerAddresses = Collections.emptyMap(); private DatastoreContext datastoreContext; - private SchemaContext schemaContext; + private SchemaContextProvider schemaContextProvider; private DatastoreSnapshot.ShardSnapshot restoreFromSnapshot; private TipProducingDataTree dataTree; private volatile boolean sealed; @@ -861,9 +933,9 @@ public class Shard extends RaftActor { return self(); } - public T schemaContext(final SchemaContext newSchemaContext) { + public T schemaContextProvider(final SchemaContextProvider schemaContextProvider) { checkSealed(); - this.schemaContext = newSchemaContext; + this.schemaContextProvider = Preconditions.checkNotNull(schemaContextProvider); return self(); } @@ -892,7 +964,7 @@ public class Shard extends RaftActor { } public SchemaContext getSchemaContext() { - return schemaContext; + return Verify.verifyNotNull(schemaContextProvider.getSchemaContext()); } public DatastoreSnapshot.ShardSnapshot getRestoreFromSnapshot() { @@ -919,7 +991,7 @@ public class Shard extends RaftActor { 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"); + Preconditions.checkNotNull(schemaContextProvider, "schemaContextProvider should not be null"); } public Props props() { @@ -938,4 +1010,8 @@ public class Shard extends RaftActor { Ticker ticker() { return Ticker.systemTicker(); } + + void scheduleNextPendingTransaction() { + self().tell(RESUME_NEXT_PENDING_TRANSACTION, ActorRef.noSender()); + } }