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=eeacdd9b6ffa9cb247c698d64c68936f8e594caa;hp=8e00aa6091adafb24c4ef2c8636cfaae7f34e091;hb=HEAD;hpb=7011afc1380ca25ad3de4cc8865f12fe3f5e09cf 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 8e00aa6091..1fcaa9d64d 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 @@ -7,32 +7,40 @@ */ package org.opendaylight.controller.cluster.datastore; +import static com.google.common.base.Preconditions.checkState; import static com.google.common.base.Verify.verify; +import static com.google.common.base.Verify.verifyNotNull; +import static java.util.Objects.requireNonNull; import akka.actor.ActorRef; import akka.actor.ActorSelection; import akka.actor.Cancellable; import akka.actor.ExtendedActorSystem; +import akka.actor.PoisonPill; import akka.actor.Props; import akka.actor.Status; import akka.actor.Status.Failure; +import akka.persistence.RecoveryCompleted; +import akka.persistence.SnapshotOffer; import akka.serialization.JavaSerializer; 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.ImmutableSet; import com.google.common.collect.Range; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; 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.Optional; +import java.util.OptionalLong; import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; import org.eclipse.jdt.annotation.NonNull; import org.eclipse.jdt.annotation.Nullable; import org.opendaylight.controller.cluster.access.ABIVersion; @@ -44,8 +52,6 @@ import org.opendaylight.controller.cluster.access.commands.OutOfSequenceEnvelope 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; import org.opendaylight.controller.cluster.access.concepts.RequestSuccess; @@ -58,13 +64,10 @@ import org.opendaylight.controller.cluster.common.actor.CommonConfig; import org.opendaylight.controller.cluster.common.actor.Dispatchers; import org.opendaylight.controller.cluster.common.actor.Dispatchers.DispatcherType; import org.opendaylight.controller.cluster.common.actor.MessageTracker; -import org.opendaylight.controller.cluster.common.actor.MessageTracker.Error; import org.opendaylight.controller.cluster.common.actor.MeteringBehavior; +import org.opendaylight.controller.cluster.datastore.actors.JsonExportActor; import org.opendaylight.controller.cluster.datastore.exceptions.NoShardLeaderException; import org.opendaylight.controller.cluster.datastore.identifiers.ShardIdentifier; -import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shard.ShardDataTreeListenerInfoMXBeanImpl; -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.AbortTransaction; import org.opendaylight.controller.cluster.datastore.messages.ActorInitialized; import org.opendaylight.controller.cluster.datastore.messages.BatchedModifications; @@ -73,7 +76,10 @@ import org.opendaylight.controller.cluster.datastore.messages.CloseTransactionCh import org.opendaylight.controller.cluster.datastore.messages.CommitTransaction; import org.opendaylight.controller.cluster.datastore.messages.CreateTransaction; import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionReply; +import org.opendaylight.controller.cluster.datastore.messages.DataTreeChangedReply; import org.opendaylight.controller.cluster.datastore.messages.ForwardedReadyTransaction; +import org.opendaylight.controller.cluster.datastore.messages.GetKnownClients; +import org.opendaylight.controller.cluster.datastore.messages.GetKnownClientsReply; import org.opendaylight.controller.cluster.datastore.messages.GetShardDataTree; import org.opendaylight.controller.cluster.datastore.messages.MakeLeaderLocal; import org.opendaylight.controller.cluster.datastore.messages.OnDemandShardState; @@ -96,18 +102,19 @@ import org.opendaylight.controller.cluster.raft.RaftActor; import org.opendaylight.controller.cluster.raft.RaftActorRecoveryCohort; import org.opendaylight.controller.cluster.raft.RaftActorSnapshotCohort; import org.opendaylight.controller.cluster.raft.RaftState; +import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry; 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.Payload; 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.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.config.distributed.datastore.provider.rev231229.DataStoreProperties.ExportOnRecovery; import org.opendaylight.yangtools.concepts.Identifier; -import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTree; -import org.opendaylight.yangtools.yang.data.api.schema.tree.DataValidationFailedException; -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 org.opendaylight.yangtools.yang.data.tree.api.DataTree; +import org.opendaylight.yangtools.yang.data.tree.api.DataValidationFailedException; +import org.opendaylight.yangtools.yang.data.tree.api.TreeType; +import org.opendaylight.yangtools.yang.model.api.EffectiveModelContext; import scala.concurrent.duration.FiniteDuration; /** @@ -116,6 +123,7 @@ import scala.concurrent.duration.FiniteDuration; *

* Our Shard uses InMemoryDataTree as it's internal representation and delegates all requests it */ +// FIXME: non-final for testing? public class Shard extends RaftActor { @VisibleForTesting @@ -146,6 +154,9 @@ public class Shard extends RaftActor { private static final Collection SUPPORTED_ABIVERSIONS; + // Make sure to keep this in sync with the journal configuration in factory-akka.conf + public static final String NON_PERSISTENT_JOURNAL_ID = "akka.persistence.non-persistent.journal"; + static { final ABIVersion[] values = ABIVersion.values(); final ABIVersion[] real = Arrays.copyOfRange(values, 1, values.length - 1); @@ -169,6 +180,7 @@ public class Shard extends RaftActor { private DatastoreContext datastoreContext; + @Deprecated(since = "9.0.0", forRemoval = true) private final ShardCommitCoordinator commitCoordinator; private long transactionCommitTimeout; @@ -179,6 +191,7 @@ public class Shard extends RaftActor { private final MessageTracker appendEntriesReplyTracker; + @Deprecated(since = "9.0.0", forRemoval = true) private final ShardTransactionActorFactory transactionActorFactory; private final ShardSnapshotCohort snapshotCohort; @@ -187,9 +200,12 @@ public class Shard extends RaftActor { private ShardSnapshot restoreFromSnapshot; + @Deprecated(since = "9.0.0", forRemoval = true) private final ShardTransactionMessageRetrySupport messageRetrySupport; - private final FrontendMetadata frontendMetadata; + @VisibleForTesting + final FrontendMetadata frontendMetadata; + private Map knownFrontends = ImmutableMap.of(); private boolean paused; @@ -198,15 +214,27 @@ public class Shard extends RaftActor { private final MessageAssembler requestMessageAssembler; - protected Shard(final AbstractBuilder builder) { + private final ExportOnRecovery exportOnRecovery; + + private final ActorRef exportActor; + + @SuppressFBWarnings(value = "MC_OVERRIDABLE_METHOD_CALL_IN_CONSTRUCTOR", justification = "Akka class design") + Shard(final AbstractBuilder builder) { super(builder.getId().toString(), builder.getPeerAddresses(), Optional.of(builder.getDatastoreContext().getShardRaftConfig()), DataStoreVersions.CURRENT_VERSION); - this.name = builder.getId().toString(); - this.shardName = builder.getId().getShardName(); - this.datastoreContext = builder.getDatastoreContext(); - this.restoreFromSnapshot = builder.getRestoreFromSnapshot(); - this.frontendMetadata = new FrontendMetadata(name); + name = builder.getId().toString(); + shardName = builder.getId().getShardName(); + datastoreContext = builder.getDatastoreContext(); + restoreFromSnapshot = builder.getRestoreFromSnapshot(); + frontendMetadata = new FrontendMetadata(name); + exportOnRecovery = datastoreContext.getExportOnRecovery(); + + exportActor = switch (exportOnRecovery) { + case Json -> getContext().actorOf(JsonExportActor.props(builder.getSchemaContext(), + datastoreContext.getRecoveryExportBaseDir())); + case Off -> null; + }; setPersistence(datastoreContext.isPersistent()); @@ -216,19 +244,21 @@ public class Shard extends RaftActor { new ShardDataTreeChangeListenerPublisherActorProxy(getContext(), name + "-DTCL-publisher", name); if (builder.getDataTree() != null) { store = new ShardDataTree(this, builder.getSchemaContext(), builder.getDataTree(), - treeChangeListenerPublisher, name, frontendMetadata); + treeChangeListenerPublisher, name, + frontendMetadata); } else { store = new ShardDataTree(this, builder.getSchemaContext(), builder.getTreeType(), - builder.getDatastoreContext().getStoreRoot(), treeChangeListenerPublisher, name, frontendMetadata); + builder.getDatastoreContext().getStoreRoot(), treeChangeListenerPublisher, name, + frontendMetadata); } - shardMBean = ShardMBeanFactory.getShardStatsMBean(name, datastoreContext.getDataStoreMXBeanType(), this); + shardMBean = ShardStats.create(name, datastoreContext.getDataStoreMXBeanType(), this); if (isMetricsCaptureEnabled()) { getContext().become(new MeteringBehavior(this)); } - commitCoordinator = new ShardCommitCoordinator(store, LOG, this.name); + commitCoordinator = new ShardCommitCoordinator(store, LOG, name); setTransactionCommitTimeout(); @@ -244,16 +274,16 @@ public class Shard extends RaftActor { self(), getContext(), shardMBean, builder.getId().getShardName()); snapshotCohort = ShardSnapshotCohort.create(getContext(), builder.getId().getMemberName(), store, LOG, - this.name); + name, datastoreContext); messageRetrySupport = new ShardTransactionMessageRetrySupport(this); - responseMessageSlicer = MessageSlicer.builder().logContext(this.name) + responseMessageSlicer = MessageSlicer.builder().logContext(name) .messageSliceSize(datastoreContext.getMaximumMessageSliceSize()) .fileBackedStreamFactory(getRaftActorContext().getFileBackedOutputStreamFactory()) .expireStateAfterInactivity(2, TimeUnit.MINUTES).build(); - requestMessageAssembler = MessageAssembler.builder().logContext(this.name) + requestMessageAssembler = MessageAssembler.builder().logContext(name) .fileBackedStreamFactory(getRaftActorContext().getFileBackedOutputStreamFactory()) .assembledMessageCallback((message, sender) -> self().tell(message, sender)) .expireStateAfterInactivity(datastoreContext.getRequestTimeout(), TimeUnit.NANOSECONDS).build(); @@ -269,13 +299,13 @@ public class Shard extends RaftActor { } private Optional createRoleChangeNotifier(final String shardId) { - ActorRef shardRoleChangeNotifier = this.getContext().actorOf( + ActorRef shardRoleChangeNotifier = getContext().actorOf( RoleChangeNotifier.getProps(shardId), shardId + "-notifier"); return Optional.of(shardRoleChangeNotifier); } @Override - public void postStop() { + public final void postStop() throws Exception { LOG.info("Stopping Shard {}", persistenceId()); super.postStop(); @@ -293,64 +323,68 @@ public class Shard extends RaftActor { } @Override - protected void handleRecover(final Object message) { + protected final void handleRecover(final Object message) { LOG.debug("{}: onReceiveRecover: Received message {} from {}", persistenceId(), message.getClass(), getSender()); super.handleRecover(message); + + switch (exportOnRecovery) { + case Json: + if (message instanceof SnapshotOffer) { + exportActor.tell(new JsonExportActor.ExportSnapshot(store.readCurrentData().orElseThrow(), name), + ActorRef.noSender()); + } else if (message instanceof ReplicatedLogEntry replicatedLogEntry) { + exportActor.tell(new JsonExportActor.ExportJournal(replicatedLogEntry), ActorRef.noSender()); + } else if (message instanceof RecoveryCompleted) { + exportActor.tell(new JsonExportActor.FinishExport(name), ActorRef.noSender()); + exportActor.tell(PoisonPill.getInstance(), ActorRef.noSender()); + } + break; + case Off: + default: + break; + } + if (LOG.isTraceEnabled()) { appendEntriesReplyTracker.begin(); } } @Override + // non-final for TestShard protected void handleNonRaftCommand(final Object message) { - try (MessageTracker.Context context = appendEntriesReplyTracker.received(message)) { - final Optional maybeError = context.error(); + try (var context = appendEntriesReplyTracker.received(message)) { + final var maybeError = context.error(); if (maybeError.isPresent()) { LOG.trace("{} : AppendEntriesReply failed to arrive at the expected interval {}", persistenceId(), - maybeError.get()); + maybeError.orElseThrow()); } store.resetTransactionBatch(); - if (message instanceof RequestEnvelope) { - handleRequestEnvelope((RequestEnvelope)message); + if (message instanceof RequestEnvelope request) { + handleRequestEnvelope(request); } else if (MessageAssembler.isHandledMessage(message)) { handleRequestAssemblerMessage(message); - } else if (message instanceof ConnectClientRequest) { - handleConnectClient((ConnectClientRequest)message); - } else if (CreateTransaction.isSerializedType(message)) { - handleCreateTransaction(message); - } else if (message instanceof BatchedModifications) { - handleBatchedModifications((BatchedModifications)message); - } else if (message instanceof ForwardedReadyTransaction) { - handleForwardedReadyTransaction((ForwardedReadyTransaction) message); - } else if (message instanceof ReadyLocalTransaction) { - handleReadyLocalTransaction((ReadyLocalTransaction)message); - } else if (CanCommitTransaction.isSerializedType(message)) { - handleCanCommitTransaction(CanCommitTransaction.fromSerializable(message)); - } else if (CommitTransaction.isSerializedType(message)) { - handleCommitTransaction(CommitTransaction.fromSerializable(message)); - } else if (AbortTransaction.isSerializedType(message)) { - handleAbortTransaction(AbortTransaction.fromSerializable(message)); - } else if (CloseTransactionChain.isSerializedType(message)) { - closeTransactionChain(CloseTransactionChain.fromSerializable(message)); - } else if (message instanceof RegisterDataTreeChangeListener) { - treeChangeSupport.onMessage((RegisterDataTreeChangeListener) message, isLeader(), hasLeader()); - } else if (message instanceof UpdateSchemaContext) { - updateSchemaContext((UpdateSchemaContext) message); - } else if (message instanceof PeerAddressResolved) { - PeerAddressResolved resolved = (PeerAddressResolved) message; + } else if (message instanceof ConnectClientRequest request) { + handleConnectClient(request); + } else if (message instanceof DataTreeChangedReply) { + // Ignore reply + } else if (message instanceof RegisterDataTreeChangeListener request) { + treeChangeSupport.onMessage(request, isLeader(), hasLeader()); + } else if (message instanceof UpdateSchemaContext request) { + updateSchemaContext(request); + } else if (message instanceof PeerAddressResolved resolved) { setPeerAddress(resolved.getPeerId(), resolved.getPeerAddress()); } else if (TX_COMMIT_TIMEOUT_CHECK_MESSAGE.equals(message)) { commitTimeoutCheck(); - } else if (message instanceof DatastoreContext) { - onDatastoreContext((DatastoreContext)message); + } else if (message instanceof DatastoreContext request) { + onDatastoreContext(request); } else if (message instanceof RegisterRoleChangeListener) { - roleChangeNotifier.get().forward(message, context()); - } else if (message instanceof FollowerInitialSyncUpStatus) { - shardMBean.setFollowerInitialSyncStatus(((FollowerInitialSyncUpStatus) message).isInitialSyncDone()); + roleChangeNotifier.orElseThrow().forward(message, context()); + } else if (message instanceof FollowerInitialSyncUpStatus request) { + shardMBean.setFollowerInitialSyncStatus(request.isInitialSyncDone()); context().parent().tell(message, self()); } else if (GET_SHARD_MBEAN_MESSAGE.equals(message)) { sender().tell(getShardMBean(), self()); @@ -358,17 +392,37 @@ public class Shard extends RaftActor { 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 if (message instanceof DataTreeCohortActorRegistry.CohortRegistryCommand) { - store.processCohortRegistryCommand(getSender(), - (DataTreeCohortActorRegistry.CohortRegistryCommand) message); + } else if (message instanceof DataTreeCohortActorRegistry.CohortRegistryCommand request) { + store.processCohortRegistryCommand(getSender(), request); } else if (message instanceof MakeLeaderLocal) { onMakeLeaderLocal(); } else if (RESUME_NEXT_PENDING_TRANSACTION.equals(message)) { store.resumeNextPendingTransaction(); + } else if (GetKnownClients.INSTANCE.equals(message)) { + handleGetKnownClients(); } else if (!responseMessageSlicer.handleMessage(message)) { - super.handleNonRaftCommand(message); + // Ask-based protocol messages + if (CreateTransaction.isSerializedType(message)) { + handleCreateTransaction(message); + } else if (message instanceof BatchedModifications request) { + handleBatchedModifications(request); + } else if (message instanceof ForwardedReadyTransaction request) { + handleForwardedReadyTransaction(request); + } else if (message instanceof ReadyLocalTransaction request) { + handleReadyLocalTransaction(request); + } else if (CanCommitTransaction.isSerializedType(message)) { + handleCanCommitTransaction(CanCommitTransaction.fromSerializable(message)); + } else if (CommitTransaction.isSerializedType(message)) { + handleCommitTransaction(CommitTransaction.fromSerializable(message)); + } else if (AbortTransaction.isSerializedType(message)) { + handleAbortTransaction(AbortTransaction.fromSerializable(message)); + } else if (CloseTransactionChain.isSerializedType(message)) { + closeTransactionChain(CloseTransactionChain.fromSerializable(message)); + } else if (ShardTransactionMessageRetrySupport.TIMER_MESSAGE_CLASS.isInstance(message)) { + messageRetrySupport.onTimerMessage(message); + } else { + super.handleNonRaftCommand(message); + } } } } @@ -413,22 +467,22 @@ public class Shard extends RaftActor { requestMessageAssembler.checkExpiredAssembledMessageState(); } - private Optional updateAccess(final SimpleShardDataTreeCohort cohort) { - final FrontendIdentifier frontend = cohort.getIdentifier().getHistoryId().getClientId().getFrontendId(); + private OptionalLong updateAccess(final SimpleShardDataTreeCohort cohort) { + final FrontendIdentifier frontend = cohort.transactionId().getHistoryId().getClientId().getFrontendId(); final LeaderFrontendState state = knownFrontends.get(frontend); if (state == null) { // Not tell-based protocol, do nothing - return Optional.absent(); + return OptionalLong.empty(); } if (isIsolatedLeader()) { // We are isolated and no new request can come through until we emerge from it. We are still updating // liveness of frontend when we see it attempting to communicate. Use the last access timer. - return Optional.of(state.getLastSeenTicks()); + return OptionalLong.of(state.getLastSeenTicks()); } // If this frontend has freshly connected, give it some time to catch up before killing its transactions. - return Optional.of(state.getLastConnectTicks()); + return OptionalLong.of(state.getLastConnectTicks()); } private void disableTracking(final DisableTrackingPayload payload) { @@ -578,14 +632,12 @@ public class Shard extends RaftActor { throw new NotLeaderException(getSelf()); } - final Request request = envelope.getMessage(); - if (request instanceof TransactionRequest) { - final TransactionRequest txReq = (TransactionRequest)request; - final ClientIdentifier clientId = txReq.getTarget().getHistoryId().getClientId(); + final var request = envelope.getMessage(); + if (request instanceof TransactionRequest txReq) { + final var clientId = txReq.getTarget().getHistoryId().getClientId(); return getFrontend(clientId).handleTransactionRequest(txReq, envelope, now); - } else if (request instanceof LocalHistoryRequest) { - final LocalHistoryRequest lhReq = (LocalHistoryRequest)request; - final ClientIdentifier clientId = lhReq.getTarget().getClientId(); + } else if (request instanceof LocalHistoryRequest lhReq) { + final var clientId = lhReq.getTarget().getClientId(); return getFrontend(clientId).handleLocalHistoryRequest(lhReq, envelope, now); } else { LOG.warn("{}: rejecting unsupported request {}", persistenceId(), request); @@ -593,36 +645,48 @@ public class Shard extends RaftActor { } } + private void handleGetKnownClients() { + final ImmutableSet clients; + if (isLeader()) { + clients = knownFrontends.values().stream() + .map(LeaderFrontendState::getIdentifier) + .collect(ImmutableSet.toImmutableSet()); + } else { + clients = frontendMetadata.getClients(); + } + sender().tell(new GetKnownClientsReply(clients), self()); + } + private boolean hasLeader() { return getLeaderId() != null; } - public int getPendingTxCommitQueueSize() { + final int getPendingTxCommitQueueSize() { return store.getQueueSize(); } - public int getCohortCacheSize() { + final int getCohortCacheSize() { return commitCoordinator.getCohortCacheSize(); } @Override - protected Optional getRoleChangeNotifier() { + protected final Optional getRoleChangeNotifier() { return roleChangeNotifier; } - String getShardName() { + final String getShardName() { return shardName; } @Override - protected LeaderStateChanged newLeaderStateChanged(final String memberId, final String leaderId, + protected final LeaderStateChanged newLeaderStateChanged(final String memberId, final String leaderId, final short leaderPayloadVersion) { return isLeader() ? new ShardLeaderStateChanged(memberId, leaderId, store.getDataTree(), leaderPayloadVersion) : new ShardLeaderStateChanged(memberId, leaderId, leaderPayloadVersion); } - protected void onDatastoreContext(final DatastoreContext context) { - datastoreContext = context; + private void onDatastoreContext(final DatastoreContext context) { + datastoreContext = verifyNotNull(context); setTransactionCommitTimeout(); @@ -632,8 +696,9 @@ public class Shard extends RaftActor { } // applyState() will be invoked once consensus is reached on the payload + // non-final for mocking void persistPayload(final Identifier id, final Payload payload, final boolean batchHint) { - boolean canSkipPayload = !hasFollowers() && !persistence().isRecoveryApplicable(); + final boolean canSkipPayload = !hasFollowers() && !persistence().isRecoveryApplicable(); if (canSkipPayload) { applyState(self(), id, payload); } else { @@ -642,13 +707,14 @@ public class Shard extends RaftActor { } } + @Deprecated(since = "9.0.0", forRemoval = true) private void handleCommitTransaction(final CommitTransaction commit) { - final TransactionIdentifier txId = commit.getTransactionId(); + final var txId = commit.getTransactionId(); if (isLeader()) { askProtocolEncountered(txId); commitCoordinator.handleCommit(txId, getSender(), this); } else { - ActorSelection leader = getLeader(); + final var leader = getLeader(); if (leader == null) { messageRetrySupport.addMessageToRetry(commit, getSender(), "Could not commit transaction " + txId); } else { @@ -658,15 +724,16 @@ public class Shard extends RaftActor { } } + @Deprecated(since = "9.0.0", forRemoval = true) private void handleCanCommitTransaction(final CanCommitTransaction canCommit) { - final TransactionIdentifier txId = canCommit.getTransactionId(); + final var txId = canCommit.getTransactionId(); LOG.debug("{}: Can committing transaction {}", persistenceId(), txId); if (isLeader()) { askProtocolEncountered(txId); commitCoordinator.handleCanCommit(txId, getSender(), this); } else { - ActorSelection leader = getLeader(); + final var leader = getLeader(); if (leader == null) { messageRetrySupport.addMessageToRetry(canCommit, getSender(), "Could not canCommit transaction " + txId); @@ -678,7 +745,8 @@ public class Shard extends RaftActor { } @SuppressWarnings("checkstyle:IllegalCatch") - protected void handleBatchedModificationsLocal(final BatchedModifications batched, final ActorRef sender) { + @Deprecated(since = "9.0.0", forRemoval = true) + private void handleBatchedModificationsLocal(final BatchedModifications batched, final ActorRef sender) { askProtocolEncountered(batched.getTransactionId()); try { @@ -690,6 +758,7 @@ public class Shard extends RaftActor { } } + @Deprecated(since = "9.0.0", forRemoval = true) private void handleBatchedModifications(final BatchedModifications batched) { // This message is sent to prepare the modifications transaction directly on the Shard as an // optimization to avoid the extra overhead of a separate ShardTransaction actor. On the last @@ -707,7 +776,7 @@ public class Shard extends RaftActor { if (isLeader() && isLeaderActive) { handleBatchedModificationsLocal(batched, getSender()); } else { - ActorSelection leader = getLeader(); + final var leader = getLeader(); if (!isLeaderActive || leader == null) { messageRetrySupport.addMessageToRetry(batched, getSender(), "Could not process BatchedModifications " + batched.getTransactionId()); @@ -716,9 +785,8 @@ public class Shard extends RaftActor { // we need to reconstruct previous BatchedModifications from the transaction // DataTreeModification, honoring the max batched modification count, and forward all the // previous BatchedModifications to the new leader. - Collection newModifications = commitCoordinator - .createForwardedBatchedModifications(batched, - datastoreContext.getShardBatchedModificationCount()); + final var newModifications = commitCoordinator.createForwardedBatchedModifications(batched, + datastoreContext.getShardBatchedModificationCount()); LOG.debug("{}: Forwarding {} BatchedModifications to leader {}", persistenceId(), newModifications.size(), leader); @@ -747,23 +815,25 @@ public class Shard extends RaftActor { } @SuppressWarnings("checkstyle:IllegalCatch") - private void handleReadyLocalTransaction(final ReadyLocalTransaction message) { - LOG.debug("{}: handleReadyLocalTransaction for {}", persistenceId(), message.getTransactionId()); + @Deprecated(since = "9.0.0", forRemoval = true) + private void handleReadyLocalTransaction(final ReadyLocalTransaction message) { + final var txId = message.getTransactionId(); + LOG.debug("{}: handleReadyLocalTransaction for {}", persistenceId(), txId); - boolean isLeaderActive = isLeaderActive(); + final var isLeaderActive = isLeaderActive(); if (isLeader() && isLeaderActive) { + askProtocolEncountered(txId); try { commitCoordinator.handleReadyLocalTransaction(message, getSender(), this); } catch (Exception e) { - LOG.error("{}: Error handling ReadyLocalTransaction for Tx {}", persistenceId(), - message.getTransactionId(), e); + LOG.error("{}: Error handling ReadyLocalTransaction for Tx {}", persistenceId(), txId, e); getSender().tell(new Failure(e), getSelf()); } } else { - ActorSelection leader = getLeader(); + final var leader = getLeader(); if (!isLeaderActive || leader == null) { messageRetrySupport.addMessageToRetry(message, getSender(), - "Could not process ready local transaction " + message.getTransactionId()); + "Could not process ready local transaction " + txId); } else { LOG.debug("{}: Forwarding ReadyLocalTransaction to leader {}", persistenceId(), leader); message.setRemoteVersion(getCurrentBehavior().getLeaderPayloadVersion()); @@ -772,22 +842,23 @@ public class Shard extends RaftActor { } } + @Deprecated(since = "9.0.0", forRemoval = true) private void handleForwardedReadyTransaction(final ForwardedReadyTransaction forwardedReady) { LOG.debug("{}: handleForwardedReadyTransaction for {}", persistenceId(), forwardedReady.getTransactionId()); - boolean isLeaderActive = isLeaderActive(); + final var isLeaderActive = isLeaderActive(); if (isLeader() && isLeaderActive) { askProtocolEncountered(forwardedReady.getTransactionId()); commitCoordinator.handleForwardedReadyTransaction(forwardedReady, getSender(), this); } else { - ActorSelection leader = getLeader(); + final var leader = getLeader(); if (!isLeaderActive || leader == null) { messageRetrySupport.addMessageToRetry(forwardedReady, getSender(), "Could not process forwarded ready transaction " + forwardedReady.getTransactionId()); } else { LOG.debug("{}: Forwarding ForwardedReadyTransaction to leader {}", persistenceId(), leader); - ReadyLocalTransaction readyLocal = new ReadyLocalTransaction(forwardedReady.getTransactionId(), + final var readyLocal = new ReadyLocalTransaction(forwardedReady.getTransactionId(), forwardedReady.getTransaction().getSnapshot(), forwardedReady.isDoImmediateCommit(), forwardedReady.getParticipatingShardNames()); readyLocal.setRemoteVersion(getCurrentBehavior().getLeaderPayloadVersion()); @@ -796,16 +867,18 @@ public class Shard extends RaftActor { } } + @Deprecated(since = "9.0.0", forRemoval = true) private void handleAbortTransaction(final AbortTransaction abort) { - final TransactionIdentifier transactionId = abort.getTransactionId(); + final var transactionId = abort.getTransactionId(); askProtocolEncountered(transactionId); doAbortTransaction(transactionId, getSender()); } - void doAbortTransaction(final Identifier transactionID, final ActorRef sender) { + final void doAbortTransaction(final Identifier transactionID, final ActorRef sender) { commitCoordinator.handleAbort(transactionID, sender, this); } + @Deprecated(since = "9.0.0", forRemoval = true) private void handleCreateTransaction(final Object message) { if (isLeader()) { createTransaction(CreateTransaction.fromSerializable(message)); @@ -817,14 +890,12 @@ public class Shard extends RaftActor { } } + @Deprecated(since = "9.0.0", forRemoval = true) private void closeTransactionChain(final CloseTransactionChain closeTransactionChain) { if (isLeader()) { - final LocalHistoryIdentifier id = closeTransactionChain.getIdentifier(); + final var id = closeTransactionChain.getIdentifier(); askProtocolEncountered(id.getClientId()); - - // FIXME: CONTROLLER-1628: stage purge once no transactions are present - store.closeTransactionChain(id, null); - store.purgeTransactionChain(id, null); + store.closeTransactionChain(id); } else if (getLeader() != null) { getLeader().forward(closeTransactionChain, getContext()); } else { @@ -832,6 +903,7 @@ public class Shard extends RaftActor { } } + @Deprecated(since = "9.0.0", forRemoval = true) @SuppressWarnings("checkstyle:IllegalCatch") private void createTransaction(final CreateTransaction createTransaction) { askProtocolEncountered(createTransaction.getTransactionId()); @@ -842,7 +914,7 @@ public class Shard extends RaftActor { return; } - ActorRef transactionActor = createTransaction(createTransaction.getTransactionType(), + final var transactionActor = createTransaction(createTransaction.getTransactionType(), createTransaction.getTransactionId()); getSender().tell(new CreateTransactionReply(Serialization.serializedActorPath(transactionActor), @@ -852,6 +924,7 @@ public class Shard extends RaftActor { } } + @Deprecated(since = "9.0.0", forRemoval = true) private ActorRef createTransaction(final int transactionType, final TransactionIdentifier transactionId) { LOG.debug("{}: Creating transaction : {} ", persistenceId(), transactionId); return transactionActorFactory.newShardTransaction(TransactionType.fromInt(transactionType), @@ -859,14 +932,16 @@ public class Shard extends RaftActor { } // Called on leader only + @Deprecated(since = "9.0.0", forRemoval = true) private void askProtocolEncountered(final TransactionIdentifier transactionId) { askProtocolEncountered(transactionId.getHistoryId().getClientId()); } // Called on leader only + @Deprecated(since = "9.0.0", forRemoval = true) private void askProtocolEncountered(final ClientIdentifier clientId) { - final FrontendIdentifier frontend = clientId.getFrontendId(); - final LeaderFrontendState state = knownFrontends.get(frontend); + final var frontend = clientId.getFrontendId(); + final var state = knownFrontends.get(frontend); if (!(state instanceof LeaderFrontendState.Disabled)) { LOG.debug("{}: encountered ask-based client {}, disabling transaction tracking", persistenceId(), clientId); if (knownFrontends.isEmpty()) { @@ -880,11 +955,11 @@ public class Shard extends RaftActor { } private void updateSchemaContext(final UpdateSchemaContext message) { - updateSchemaContext(message.getSchemaContext()); + updateSchemaContext(message.modelContext()); } @VisibleForTesting - void updateSchemaContext(final SchemaContext schemaContext) { + void updateSchemaContext(final @NonNull EffectiveModelContext schemaContext) { store.updateSchemaContext(schemaContext); } @@ -894,13 +969,12 @@ public class Shard extends RaftActor { } @Override - @VisibleForTesting - public RaftActorSnapshotCohort getRaftActorSnapshotCohort() { + protected final RaftActorSnapshotCohort getRaftActorSnapshotCohort() { return snapshotCohort; } @Override - protected RaftActorRecoveryCohort getRaftActorRecoveryCohort() { + protected final RaftActorRecoveryCohort getRaftActorRecoveryCohort() { if (restoreFromSnapshot == null) { return ShardRecoveryCoordinator.create(store, persistenceId(), LOG); } @@ -909,17 +983,18 @@ public class Shard extends RaftActor { } @Override + // non-final for testing protected void onRecoveryComplete() { restoreFromSnapshot = null; //notify shard manager - getContext().parent().tell(new ActorInitialized(), getSelf()); + getContext().parent().tell(new ActorInitialized(getSelf()), ActorRef.noSender()); // Being paranoid here - this method should only be called once but just in case... if (txCommitTimeoutCheckSchedule == null) { // Schedule a message to be periodically sent to check if the current in-progress // transaction should be expired and aborted. - FiniteDuration period = FiniteDuration.create(transactionCommitTimeout / 3, TimeUnit.MILLISECONDS); + final var period = FiniteDuration.create(transactionCommitTimeout / 3, TimeUnit.MILLISECONDS); txCommitTimeoutCheckSchedule = getContext().system().scheduler().schedule( period, period, getSelf(), TX_COMMIT_TIMEOUT_CHECK_MESSAGE, getContext().dispatcher(), ActorRef.noSender()); @@ -927,15 +1002,15 @@ public class Shard extends RaftActor { } @Override - protected void applyState(final ActorRef clientActor, final Identifier identifier, final Object data) { - if (data instanceof Payload) { - if (data instanceof DisableTrackingPayload) { - disableTracking((DisableTrackingPayload) data); + protected final void applyState(final ActorRef clientActor, final Identifier identifier, final Object data) { + if (data instanceof Payload payload) { + if (payload instanceof DisableTrackingPayload disableTracking) { + disableTracking(disableTracking); return; } try { - store.applyReplicatedPayload(identifier, (Payload)data); + store.applyReplicatedPayload(identifier, payload); } catch (DataValidationFailedException | IOException e) { LOG.error("{}: Error applying replica {}", persistenceId(), identifier, e); } @@ -945,7 +1020,7 @@ public class Shard extends RaftActor { } @Override - protected void onStateChanged() { + protected final void onStateChanged() { boolean isLeader = isLeader(); boolean hasLeader = hasLeader(); treeChangeSupport.onLeadershipChange(isLeader, hasLeader); @@ -968,7 +1043,7 @@ public class Shard extends RaftActor { } @Override - protected void onLeaderChanged(final String oldLeader, final String newLeader) { + protected final void onLeaderChanged(final String oldLeader, final String newLeader) { shardMBean.incrementLeadershipChangeCount(); paused = false; @@ -989,7 +1064,9 @@ public class Shard extends RaftActor { // them to transaction messages and send to the new leader. ActorSelection leader = getLeader(); if (leader != null) { - Collection messagesToForward = convertPendingTransactionsToMessages(); + // Clears all pending transactions and converts them to messages to be forwarded to a new leader. + Collection messagesToForward = commitCoordinator.convertPendingTransactionsToMessages( + datastoreContext.getShardBatchedModificationCount()); if (!messagesToForward.isEmpty()) { LOG.debug("{}: Forwarding {} pending transaction messages to leader {}", persistenceId(), @@ -1007,7 +1084,7 @@ public class Shard extends RaftActor { } } else { // We have become the leader, we need to reconstruct frontend state - knownFrontends = Verify.verifyNotNull(frontendMetadata.toLeaderState(this)); + knownFrontends = verifyNotNull(frontendMetadata.toLeaderState(this)); LOG.debug("{}: became leader with frontend state for {}", persistenceId(), knownFrontends.keySet()); } @@ -1016,67 +1093,68 @@ public class Shard extends RaftActor { } } - /** - * Clears all pending transactions and converts them to messages to be forwarded to a new leader. - * - * @return the converted messages - */ - public Collection convertPendingTransactionsToMessages() { - return commitCoordinator.convertPendingTransactionsToMessages( - datastoreContext.getShardBatchedModificationCount()); - } - @Override - protected void pauseLeader(final Runnable operation) { + protected final void pauseLeader(final Runnable operation) { LOG.debug("{}: In pauseLeader, operation: {}", persistenceId(), operation); paused = true; // Tell-based protocol can replay transaction state, so it is safe to blow it up when we are paused. - if (datastoreContext.isUseTellBasedProtocol()) { - knownFrontends.values().forEach(LeaderFrontendState::retire); - knownFrontends = ImmutableMap.of(); - } + knownFrontends.values().forEach(LeaderFrontendState::retire); + knownFrontends = ImmutableMap.of(); store.setRunOnPendingTransactionsComplete(operation); } @Override - protected void unpauseLeader() { + protected final void unpauseLeader() { LOG.debug("{}: In unpauseLeader", persistenceId()); paused = false; store.setRunOnPendingTransactionsComplete(null); // Restore tell-based protocol state as if we were becoming the leader - knownFrontends = Verify.verifyNotNull(frontendMetadata.toLeaderState(this)); + knownFrontends = verifyNotNull(frontendMetadata.toLeaderState(this)); } @Override - protected OnDemandRaftState.AbstractBuilder newOnDemandRaftStateBuilder() { - return OnDemandShardState.newBuilder().treeChangeListenerActors(treeChangeSupport.getListenerActors()) - .commitCohortActors(store.getCohortActors()); + protected final OnDemandRaftState.AbstractBuilder newOnDemandRaftStateBuilder() { + return OnDemandShardState.newBuilder() + .treeChangeListenerActors(treeChangeSupport.getListenerActors()) + .commitCohortActors(store.getCohortActors()); } @Override - public String persistenceId() { - return this.name; + public final String persistenceId() { + return name; + } + + @Override + public final String journalPluginId() { + // This method may be invoked from super constructor (wonderful), hence we also need to handle the case of + // the field being uninitialized because our constructor is not finished. + if (datastoreContext != null && !datastoreContext.isPersistent()) { + return NON_PERSISTENT_JOURNAL_ID; + } + return super.journalPluginId(); } @VisibleForTesting - ShardCommitCoordinator getCommitCoordinator() { + final ShardCommitCoordinator getCommitCoordinator() { return commitCoordinator; } - public DatastoreContext getDatastoreContext() { + // non-final for mocking + DatastoreContext getDatastoreContext() { return datastoreContext; } @VisibleForTesting - public ShardDataTree getDataStore() { + final ShardDataTree getDataStore() { return store; } @VisibleForTesting + // non-final for mocking ShardStats getShardMBean() { return shardMBean; } @@ -1086,21 +1164,22 @@ public class Shard extends RaftActor { } public abstract static class AbstractBuilder, S extends Shard> { - private final Class shardClass; + private final Class shardClass; private ShardIdentifier id; private Map peerAddresses = Collections.emptyMap(); private DatastoreContext datastoreContext; - private SchemaContextProvider schemaContextProvider; + private Supplier<@NonNull EffectiveModelContext> schemaContextProvider; private DatastoreSnapshot.ShardSnapshot restoreFromSnapshot; private DataTree dataTree; + private volatile boolean sealed; - protected AbstractBuilder(final Class shardClass) { + AbstractBuilder(final Class shardClass) { this.shardClass = shardClass; } - protected void checkSealed() { - Preconditions.checkState(!sealed, "Builder isalready sealed - further modifications are not allowed"); + final void checkSealed() { + checkState(!sealed, "Builder is already sealed - further modifications are not allowed"); } @SuppressWarnings("unchecked") @@ -1110,37 +1189,37 @@ public class Shard extends RaftActor { public T id(final ShardIdentifier newId) { checkSealed(); - this.id = newId; + id = newId; return self(); } public T peerAddresses(final Map newPeerAddresses) { checkSealed(); - this.peerAddresses = newPeerAddresses; + peerAddresses = newPeerAddresses; return self(); } public T datastoreContext(final DatastoreContext newDatastoreContext) { checkSealed(); - this.datastoreContext = newDatastoreContext; + datastoreContext = newDatastoreContext; return self(); } - public T schemaContextProvider(final SchemaContextProvider newSchemaContextProvider) { + public T schemaContextProvider(final Supplier<@NonNull EffectiveModelContext> newSchemaContextProvider) { checkSealed(); - this.schemaContextProvider = Preconditions.checkNotNull(newSchemaContextProvider); + schemaContextProvider = requireNonNull(newSchemaContextProvider); return self(); } public T restoreFromSnapshot(final DatastoreSnapshot.ShardSnapshot newRestoreFromSnapshot) { checkSealed(); - this.restoreFromSnapshot = newRestoreFromSnapshot; + restoreFromSnapshot = newRestoreFromSnapshot; return self(); } public T dataTree(final DataTree newDataTree) { checkSealed(); - this.dataTree = newDataTree; + dataTree = newDataTree; return self(); } @@ -1156,8 +1235,8 @@ public class Shard extends RaftActor { return datastoreContext; } - public SchemaContext getSchemaContext() { - return Verify.verifyNotNull(schemaContextProvider.getSchemaContext()); + public EffectiveModelContext getSchemaContext() { + return verifyNotNull(schemaContextProvider.get()); } public DatastoreSnapshot.ShardSnapshot getRestoreFromSnapshot() { @@ -1169,22 +1248,17 @@ public class Shard extends RaftActor { } public TreeType getTreeType() { - switch (datastoreContext.getLogicalStoreType()) { - case CONFIGURATION: - return TreeType.CONFIGURATION; - case OPERATIONAL: - return TreeType.OPERATIONAL; - default: - throw new IllegalStateException("Unhandled logical store type " - + datastoreContext.getLogicalStoreType()); - } + return switch (datastoreContext.getLogicalStoreType()) { + case CONFIGURATION -> TreeType.CONFIGURATION; + case OPERATIONAL -> TreeType.OPERATIONAL; + }; } 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(schemaContextProvider, "schemaContextProvider should not be null"); + requireNonNull(id, "id should not be null"); + requireNonNull(peerAddresses, "peerAddresses should not be null"); + requireNonNull(datastoreContext, "dataStoreContext should not be null"); + requireNonNull(schemaContextProvider, "schemaContextProvider should not be null"); } public Props props() { @@ -1196,7 +1270,11 @@ public class Shard extends RaftActor { public static class Builder extends AbstractBuilder { Builder() { - super(Shard.class); + this(Shard.class); + } + + Builder(final Class shardClass) { + super(shardClass); } }