Speed up root overwrite check
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / ShardDataTree.java
index 3665fad559584db6eab228aecdb7c9920df9f5d3..b3688a87fa2baf0c71f46d47292cd8818525fc80 100644 (file)
@@ -7,15 +7,16 @@
  */
 package org.opendaylight.controller.cluster.datastore;
 
+import static akka.actor.ActorRef.noSender;
 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 static java.util.Objects.requireNonNullElse;
 
 import akka.actor.ActorRef;
 import akka.util.Timeout;
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.MoreObjects;
 import com.google.common.base.Stopwatch;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
@@ -50,7 +51,6 @@ import org.opendaylight.controller.cluster.access.concepts.LocalHistoryIdentifie
 import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier;
 import org.opendaylight.controller.cluster.datastore.DataTreeCohortActorRegistry.CohortRegistryCommand;
 import org.opendaylight.controller.cluster.datastore.ShardDataTreeCohort.State;
-import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shard.ShardStats;
 import org.opendaylight.controller.cluster.datastore.node.utils.transformer.ReusableNormalizedNodePruner;
 import org.opendaylight.controller.cluster.datastore.persisted.AbortTransactionPayload;
 import org.opendaylight.controller.cluster.datastore.persisted.AbstractIdentifiablePayload;
@@ -67,6 +67,7 @@ import org.opendaylight.controller.cluster.datastore.persisted.ShardDataTreeSnap
 import org.opendaylight.controller.cluster.datastore.persisted.ShardSnapshotState;
 import org.opendaylight.controller.cluster.datastore.utils.DataTreeModificationOutput;
 import org.opendaylight.controller.cluster.datastore.utils.PruningDataTreeModification;
+import org.opendaylight.controller.cluster.raft.base.messages.InitiateCaptureSnapshot;
 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
 import org.opendaylight.mdsal.common.api.OptimisticLockFailedException;
 import org.opendaylight.mdsal.common.api.TransactionCommitFailedException;
@@ -85,10 +86,12 @@ import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeModification
 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeSnapshot;
 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeTip;
 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.data.codec.binfmt.NormalizedNodeStreamVersion;
 import org.opendaylight.yangtools.yang.data.impl.schema.tree.InMemoryDataTreeFactory;
 import org.opendaylight.yangtools.yang.data.util.DataSchemaContextTree;
+import org.opendaylight.yangtools.yang.model.api.EffectiveModelContext;
 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -159,7 +162,7 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
 
     private int currentTransactionBatch;
 
-    ShardDataTree(final Shard shard, final SchemaContext schemaContext, final DataTree dataTree,
+    ShardDataTree(final Shard shard, final EffectiveModelContext schemaContext, final DataTree dataTree,
             final ShardDataTreeChangeListenerPublisher treeChangeListenerPublisher,
             final String logContext,
             final ShardDataTreeMetadata<?>... metadata) {
@@ -173,7 +176,7 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
         tip = dataTree;
     }
 
-    ShardDataTree(final Shard shard, final SchemaContext schemaContext, final TreeType treeType,
+    ShardDataTree(final Shard shard, final EffectiveModelContext schemaContext, final TreeType treeType,
             final YangInstanceIdentifier root,
             final ShardDataTreeChangeListenerPublisher treeChangeListenerPublisher,
             final String logContext,
@@ -191,7 +194,7 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
     }
 
     @VisibleForTesting
-    public ShardDataTree(final Shard shard, final SchemaContext schemaContext, final TreeType treeType) {
+    public ShardDataTree(final Shard shard, final EffectiveModelContext schemaContext, final TreeType treeType) {
         this(shard, schemaContext, treeType, YangInstanceIdentifier.empty(),
                 new DefaultShardDataTreeChangeListenerPublisher(""), "");
     }
@@ -212,9 +215,9 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
         return schemaContext;
     }
 
-    void updateSchemaContext(final SchemaContext newSchemaContext) {
-        dataTree.setSchemaContext(newSchemaContext);
-        this.schemaContext = requireNonNull(newSchemaContext);
+    void updateSchemaContext(final @NonNull EffectiveModelContext newSchemaContext) {
+        dataTree.setEffectiveModelContext(newSchemaContext);
+        this.schemaContext = newSchemaContext;
         this.dataSchemaContext = DataSchemaContextTree.from(newSchemaContext);
     }
 
@@ -438,6 +441,10 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
                     applyReplicatedCandidate((CommitTransactionPayload) payload);
                 }
             }
+
+            // make sure acquireCandidate() is the last call touching the payload data as we want it to be GC-ed.
+            checkRootOverwrite(((CommitTransactionPayload) payload).acquireCandidate().getValue()
+                    .getCandidate());
         } else if (payload instanceof AbortTransactionPayload) {
             if (identifier != null) {
                 payloadReplicationComplete((AbortTransactionPayload) payload);
@@ -468,6 +475,25 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
         }
     }
 
+    private void checkRootOverwrite(final DataTreeCandidate candidate) {
+        final DatastoreContext datastoreContext = shard.getDatastoreContext();
+        if (!datastoreContext.isSnapshotOnRootOverwrite()) {
+            return;
+        }
+
+        if (!datastoreContext.isPersistent()) {
+            // FIXME: why don't we want a snapshot in non-persistent state?
+            return;
+        }
+
+        // top level container ie "/"
+        if (candidate.getRootPath().isEmpty()
+                && candidate.getRootNode().getModificationType() == ModificationType.WRITE) {
+            LOG.debug("{}: shard root overwritten, enqueuing snapshot", logContext);
+            shard.self().tell(new InitiateCaptureSnapshot(), noSender());
+        }
+    }
+
     private void replicatePayload(final Identifier id, final Payload payload, final @Nullable Runnable callback) {
         if (callback != null) {
             replicationCallbacks.put(payload, callback);
@@ -1254,7 +1280,7 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
             return false;
         }
 
-        DataTreeTip newTip = MoreObjects.firstNonNull(first.cohort.getCandidate(), dataTree);
+        DataTreeTip newTip = requireNonNullElse(first.cohort.getCandidate(), dataTree);
         while (it.hasNext()) {
             final CommitEntry e = it.next();
             if (cohort.equals(e.cohort)) {
@@ -1267,7 +1293,7 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
 
                 return true;
             } else {
-                newTip = MoreObjects.firstNonNull(e.cohort.getCandidate(), newTip);
+                newTip = requireNonNullElse(e.cohort.getCandidate(), newTip);
             }
         }