X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?a=blobdiff_plain;ds=sidebyside;f=opendaylight%2Fmd-sal%2Fsal-distributed-datastore%2Fsrc%2Fmain%2Fjava%2Forg%2Fopendaylight%2Fcontroller%2Fcluster%2Fdatastore%2FShard.java;h=357df4ab8d819fbb7c5d46a0ea80ec025aa71200;hb=546cd1fd100dbaa36908b22c2f422320dbd8c4b2;hp=5fa9192ac0d4c727a72f2d3af8904b021c46aa80;hpb=e32959e0bbc326f47c30ed7347f9a9af26813f89;p=controller.git 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 5fa9192ac0..357df4ab8d 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 @@ -9,20 +9,23 @@ 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.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; @@ -63,11 +66,9 @@ import org.opendaylight.controller.cluster.common.actor.Dispatchers.DispatcherTy 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; @@ -76,6 +77,7 @@ 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; @@ -101,18 +103,20 @@ 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.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.rev140612.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.model.api.EffectiveModelContext; +import org.opendaylight.yangtools.yang.model.api.EffectiveModelContextProvider; import scala.concurrent.duration.FiniteDuration; /** @@ -121,6 +125,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 @@ -151,6 +156,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); @@ -205,7 +213,11 @@ public class Shard extends RaftActor { private final MessageAssembler requestMessageAssembler; - protected Shard(final AbstractBuilder builder) { + private final ExportOnRecovery exportOnRecovery; + + private final ActorRef exportActor; + + Shard(final AbstractBuilder builder) { super(builder.getId().toString(), builder.getPeerAddresses(), Optional.of(builder.getDatastoreContext().getShardRaftConfig()), DataStoreVersions.CURRENT_VERSION); @@ -214,6 +226,18 @@ public class Shard extends RaftActor { this.datastoreContext = builder.getDatastoreContext(); this.restoreFromSnapshot = builder.getRestoreFromSnapshot(); this.frontendMetadata = new FrontendMetadata(name); + this.exportOnRecovery = datastoreContext.getExportOnRecovery(); + + switch (exportOnRecovery) { + case Json: + exportActor = getContext().actorOf(JsonExportActor.props(builder.getSchemaContext(), + datastoreContext.getRecoveryExportBaseDir())); + break; + case Off: + default: + exportActor = null; + break; + } setPersistence(datastoreContext.isPersistent()); @@ -223,13 +247,15 @@ 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)); @@ -251,7 +277,7 @@ public class Shard extends RaftActor { self(), getContext(), shardMBean, builder.getId().getShardName()); snapshotCohort = ShardSnapshotCohort.create(getContext(), builder.getId().getMemberName(), store, LOG, - this.name); + this.name, datastoreContext); messageRetrySupport = new ShardTransactionMessageRetrySupport(this); @@ -282,7 +308,7 @@ public class Shard extends RaftActor { } @Override - public void postStop() throws Exception { + public final void postStop() throws Exception { LOG.info("Stopping Shard {}", persistenceId()); super.postStop(); @@ -300,17 +326,37 @@ 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().get(), name), + ActorRef.noSender()); + } else if (message instanceof ReplicatedLogEntry) { + exportActor.tell(new JsonExportActor.ExportJournal((ReplicatedLogEntry) message), + 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(); @@ -343,6 +389,8 @@ public class Shard extends RaftActor { handleAbortTransaction(AbortTransaction.fromSerializable(message)); } else if (CloseTransactionChain.isSerializedType(message)) { closeTransactionChain(CloseTransactionChain.fromSerializable(message)); + } else if (message instanceof DataTreeChangedReply) { + // Ignore reply } else if (message instanceof RegisterDataTreeChangeListener) { treeChangeSupport.onMessage((RegisterDataTreeChangeListener) message, isLeader(), hasLeader()); } else if (message instanceof UpdateSchemaContext) { @@ -618,32 +666,32 @@ public class Shard extends RaftActor { 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(); @@ -653,8 +701,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 { @@ -699,7 +748,7 @@ public class Shard extends RaftActor { } @SuppressWarnings("checkstyle:IllegalCatch") - protected void handleBatchedModificationsLocal(final BatchedModifications batched, final ActorRef sender) { + private void handleBatchedModificationsLocal(final BatchedModifications batched, final ActorRef sender) { askProtocolEncountered(batched.getTransactionId()); try { @@ -823,7 +872,7 @@ public class Shard extends RaftActor { 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); } @@ -898,11 +947,11 @@ public class Shard extends RaftActor { } private void updateSchemaContext(final UpdateSchemaContext message) { - updateSchemaContext(message.getSchemaContext()); + updateSchemaContext(message.getEffectiveModelContext()); } @VisibleForTesting - void updateSchemaContext(final SchemaContext schemaContext) { + void updateSchemaContext(final @NonNull EffectiveModelContext schemaContext) { store.updateSchemaContext(schemaContext); } @@ -912,13 +961,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); } @@ -927,6 +975,7 @@ public class Shard extends RaftActor { } @Override + // non-final for testing protected void onRecoveryComplete() { restoreFromSnapshot = null; @@ -945,7 +994,7 @@ public class Shard extends RaftActor { } @Override - protected void applyState(final ActorRef clientActor, final Identifier identifier, final Object data) { + protected final void applyState(final ActorRef clientActor, final Identifier identifier, final Object data) { if (data instanceof Payload) { if (data instanceof DisableTrackingPayload) { disableTracking((DisableTrackingPayload) data); @@ -963,7 +1012,7 @@ public class Shard extends RaftActor { } @Override - protected void onStateChanged() { + protected final void onStateChanged() { boolean isLeader = isLeader(); boolean hasLeader = hasLeader(); treeChangeSupport.onLeadershipChange(isLeader, hasLeader); @@ -986,7 +1035,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; @@ -1007,7 +1056,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(), @@ -1025,7 +1076,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()); } @@ -1034,18 +1085,8 @@ 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; @@ -1059,42 +1100,55 @@ public class Shard extends RaftActor { } @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() { + public final String persistenceId() { return this.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; } @@ -1108,18 +1162,18 @@ public class Shard extends RaftActor { private ShardIdentifier id; private Map peerAddresses = Collections.emptyMap(); private DatastoreContext datastoreContext; - private SchemaContextProvider schemaContextProvider; + private EffectiveModelContextProvider 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() { - 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") @@ -1145,7 +1199,7 @@ public class Shard extends RaftActor { return self(); } - public T schemaContextProvider(final SchemaContextProvider newSchemaContextProvider) { + public T schemaContextProvider(final EffectiveModelContextProvider newSchemaContextProvider) { checkSealed(); this.schemaContextProvider = requireNonNull(newSchemaContextProvider); return self(); @@ -1175,8 +1229,8 @@ public class Shard extends RaftActor { return datastoreContext; } - public SchemaContext getSchemaContext() { - return Verify.verifyNotNull(schemaContextProvider.getSchemaContext()); + public EffectiveModelContext getSchemaContext() { + return verifyNotNull(schemaContextProvider.getEffectiveModelContext()); } public DatastoreSnapshot.ShardSnapshot getRestoreFromSnapshot() {