Remove ReactorStmtCtx.parentStmtCtx()
[yangtools.git] / yang / yang-parser-reactor / src / main / java / org / opendaylight / yangtools / yang / parser / stmt / reactor / StatementContextBase.java
index 1e25c32305cd4ad3aa64197bc9117fc6570d37c8..50198334ce4187d87d7a2c393b90283f5ea86b2d 100644 (file)
@@ -30,9 +30,9 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.Objects;
 import java.util.Optional;
 import java.util.Set;
+import java.util.stream.Stream;
 import org.eclipse.jdt.annotation.NonNull;
 import org.eclipse.jdt.annotation.Nullable;
 import org.opendaylight.yangtools.yang.common.QName;
@@ -43,9 +43,8 @@ import org.opendaylight.yangtools.yang.model.api.meta.DeclaredStatement;
 import org.opendaylight.yangtools.yang.model.api.meta.EffectiveStatement;
 import org.opendaylight.yangtools.yang.model.api.meta.IdentifierNamespace;
 import org.opendaylight.yangtools.yang.model.api.meta.StatementDefinition;
-import org.opendaylight.yangtools.yang.model.api.meta.StatementSource;
 import org.opendaylight.yangtools.yang.model.api.stmt.AugmentStatement;
-import org.opendaylight.yangtools.yang.model.api.stmt.ConfigStatement;
+import org.opendaylight.yangtools.yang.model.api.stmt.ConfigEffectiveStatement;
 import org.opendaylight.yangtools.yang.model.api.stmt.DeviationStatement;
 import org.opendaylight.yangtools.yang.model.api.stmt.RefineStatement;
 import org.opendaylight.yangtools.yang.model.api.stmt.SchemaNodeIdentifier;
@@ -64,8 +63,8 @@ import org.opendaylight.yangtools.yang.parser.spi.meta.NamespaceKeyCriterion;
 import org.opendaylight.yangtools.yang.parser.spi.meta.NamespaceNotAvailableException;
 import org.opendaylight.yangtools.yang.parser.spi.meta.StatementNamespace;
 import org.opendaylight.yangtools.yang.parser.spi.meta.StatementSupport;
+import org.opendaylight.yangtools.yang.parser.spi.meta.StatementSupport.CopyPolicy;
 import org.opendaylight.yangtools.yang.parser.spi.meta.StmtContext;
-import org.opendaylight.yangtools.yang.parser.spi.meta.StmtContext.Mutable;
 import org.opendaylight.yangtools.yang.parser.spi.meta.StmtContextUtils;
 import org.opendaylight.yangtools.yang.parser.spi.source.ImplicitSubstatement;
 import org.opendaylight.yangtools.yang.parser.spi.source.SourceException;
@@ -84,7 +83,7 @@ import org.slf4j.LoggerFactory;
  * @param <E> Effective Statement representation
  */
 public abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E extends EffectiveStatement<A, D>>
-        extends NamespaceStorageSupport implements Mutable<A, D, E> {
+        extends ReactorStmtCtx<A, D, E> {
     /**
      * Event listener when an item is added to model namespace.
      */
@@ -140,7 +139,7 @@ public abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E
 
     private Multimap<ModelProcessingPhase, OnPhaseFinished> phaseListeners = ImmutableMultimap.of();
     private Multimap<ModelProcessingPhase, ContextMutation> phaseMutation = ImmutableMultimap.of();
-    private List<Mutable<?, ?, ?>> effective = ImmutableList.of();
+
     private List<StmtContext<?, ?, ?>> effectOfStatement = ImmutableList.of();
 
     private @Nullable ModelProcessingPhase completedPhase;
@@ -265,9 +264,6 @@ public abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E
         this.completedPhase = completedPhase;
     }
 
-    @Override
-    public abstract StatementContextBase<?, ?, ?> getParentContext();
-
     /**
      * Returns the model root for this statement.
      *
@@ -311,11 +307,6 @@ public abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E
         return getRoot().isEnabledSemanticVersioningImpl();
     }
 
-    @Override
-    public StatementSource getStatementSource() {
-        return getStatementSourceReference().getStatementSource();
-    }
-
     @Override
     public final <K, V, N extends IdentifierNamespace<K, V>> Map<K, V> getAllFromCurrentStmtCtxNamespace(
             final Class<N> type) {
@@ -342,7 +333,7 @@ public abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E
      */
     @Override
     public final <K, V, T extends K, U extends V, N extends IdentifierNamespace<K, V>> void addToNs(
-            final Class<N> type, final T key, final U value) {
+            final Class<@NonNull N> type, final T key, final U value) {
         addToNamespace(type, key, value);
     }
 
@@ -362,40 +353,38 @@ public abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E
      * @throws NamespaceNotAvailableException when the namespace is not available.
      */
     @Override
-    public final <K, V, T extends K, N extends IdentifierNamespace<K, V>> V getFromNamespace(final Class<N> type,
-            final T key) {
+    public final <K, V, T extends K, N extends IdentifierNamespace<K, V>> V getFromNamespace(
+            final Class<@NonNull N> type, final T key) {
         return getBehaviourRegistry().getNamespaceBehaviour(type).getFrom(this, key);
     }
 
-    @Override
-    public Collection<? extends Mutable<?, ?, ?>> mutableEffectiveSubstatements() {
-        if (effective instanceof ImmutableCollection) {
-            return effective;
-        }
-
-        return Collections.unmodifiableCollection(effective);
+    static final Collection<? extends Mutable<?, ?, ?>> mutableEffectiveSubstatements(
+            final List<StatementContextBase<?, ?, ?>> effective) {
+        return effective instanceof ImmutableCollection ? effective : Collections.unmodifiableCollection(effective);
     }
 
-    private void shrinkEffective() {
-        if (effective.isEmpty()) {
-            effective = ImmutableList.of();
-        }
+    private static List<StatementContextBase<?, ?, ?>> shrinkEffective(
+            final List<StatementContextBase<?, ?, ?>> effective) {
+        return effective.isEmpty() ? ImmutableList.of() : effective;
     }
 
-    public void removeStatementFromEffectiveSubstatements(final StatementDefinition statementDef) {
+    public abstract void removeStatementFromEffectiveSubstatements(StatementDefinition statementDef);
+
+    static final List<StatementContextBase<?, ?, ?>> removeStatementFromEffectiveSubstatements(
+            final List<StatementContextBase<?, ?, ?>> effective, final StatementDefinition statementDef) {
         if (effective.isEmpty()) {
-            return;
+            return effective;
         }
 
         final Iterator<? extends StmtContext<?, ?, ?>> iterator = effective.iterator();
         while (iterator.hasNext()) {
             final StmtContext<?, ?, ?> next = iterator.next();
-            if (statementDef.equals(next.getPublicDefinition())) {
+            if (statementDef.equals(next.publicDefinition())) {
                 iterator.remove();
             }
         }
 
-        shrinkEffective();
+        return shrinkEffective(effective);
     }
 
     /**
@@ -410,25 +399,29 @@ public abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E
      * @param statementDef statement definition of the statement context to remove
      * @param statementArg statement argument of the statement context to remove
      */
-    public void removeStatementFromEffectiveSubstatements(final StatementDefinition statementDef,
+    public abstract void removeStatementFromEffectiveSubstatements(StatementDefinition statementDef,
+            String statementArg);
+
+    static final List<StatementContextBase<?, ?, ?>> removeStatementFromEffectiveSubstatements(
+            final List<StatementContextBase<?, ?, ?>> effective, final StatementDefinition statementDef,
             final String statementArg) {
         if (statementArg == null) {
-            removeStatementFromEffectiveSubstatements(statementDef);
+            return removeStatementFromEffectiveSubstatements(effective, statementDef);
         }
 
         if (effective.isEmpty()) {
-            return;
+            return effective;
         }
 
-        final Iterator<Mutable<?, ?, ?>> iterator = effective.iterator();
+        final Iterator<StatementContextBase<?, ?, ?>> iterator = effective.iterator();
         while (iterator.hasNext()) {
             final Mutable<?, ?, ?> next = iterator.next();
-            if (statementDef.equals(next.getPublicDefinition()) && statementArg.equals(next.rawStatementArgument())) {
+            if (statementDef.equals(next.publicDefinition()) && statementArg.equals(next.rawArgument())) {
                 iterator.remove();
             }
         }
 
-        shrinkEffective();
+        return shrinkEffective(effective);
     }
 
     // YANG example: RPC/action statements always have 'input' and 'output' defined
@@ -438,7 +431,7 @@ public abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E
         // FIXME: YANGTOOLS-652: This does not need to be a SubstatementContext, in can be a specialized
         //                       StatementContextBase subclass.
         final Mutable<X, Y, Z> ret = new SubstatementContext<>(this, new StatementDefinitionContext<>(support),
-                ImplicitSubstatement.of(getStatementSourceReference()), rawArg);
+                ImplicitSubstatement.of(sourceReference()), rawArg);
         support.onStatementAdded(ret);
         addEffectiveSubstatement(ret);
         return ret;
@@ -448,14 +441,23 @@ public abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E
      * Adds an effective statement to collection of substatements.
      *
      * @param substatement substatement
-     * @throws IllegalStateException
-     *             if added in declared phase
-     * @throws NullPointerException
-     *             if statement parameter is null
+     * @throws IllegalStateException if added in declared phase
+     * @throws NullPointerException if statement parameter is null
      */
-    public void addEffectiveSubstatement(final Mutable<?, ?, ?> substatement) {
-        beforeAddEffectiveStatement(1);
-        effective.add(substatement);
+    public abstract void addEffectiveSubstatement(Mutable<?, ?, ?> substatement);
+
+    final List<StatementContextBase<?, ?, ?>> addEffectiveSubstatement(
+            final List<StatementContextBase<?, ?, ?>> effective, final Mutable<?, ?, ?> substatement) {
+        verifyStatement(substatement);
+
+        final List<StatementContextBase<?, ?, ?>> resized = beforeAddEffectiveStatement(effective, 1);
+        final StatementContextBase<?, ?, ?> stmt = (StatementContextBase<?, ?, ?>) substatement;
+        final ModelProcessingPhase phase = completedPhase;
+        if (phase != null) {
+            ensureCompletedPhase(stmt, phase);
+        }
+        resized.add(stmt);
+        return resized;
     }
 
     /**
@@ -467,89 +469,172 @@ public abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E
      * @throws NullPointerException
      *             if statement parameter is null
      */
-    public void addEffectiveSubstatements(final Collection<? extends Mutable<?, ?, ?>> statements) {
-        if (statements.isEmpty()) {
-            return;
+    public final void addEffectiveSubstatements(final Collection<? extends Mutable<?, ?, ?>> statements) {
+        if (!statements.isEmpty()) {
+            statements.forEach(StatementContextBase::verifyStatement);
+            addEffectiveSubstatementsImpl(statements);
+        }
+    }
+
+    abstract void addEffectiveSubstatementsImpl(Collection<? extends Mutable<?, ?, ?>> statements);
+
+    final List<StatementContextBase<?, ?, ?>> addEffectiveSubstatementsImpl(
+            final List<StatementContextBase<?, ?, ?>> effective,
+            final Collection<? extends Mutable<?, ?, ?>> statements) {
+        final List<StatementContextBase<?, ?, ?>> resized = beforeAddEffectiveStatement(effective, statements.size());
+        final Collection<? extends StatementContextBase<?, ?, ?>> casted =
+            (Collection<? extends StatementContextBase<?, ?, ?>>) statements;
+        final ModelProcessingPhase phase = completedPhase;
+        if (phase != null) {
+            for (StatementContextBase<?, ?, ?> stmt : casted) {
+                ensureCompletedPhase(stmt, phase);
+            }
+        }
+
+        resized.addAll(casted);
+        return resized;
+    }
+
+    abstract Iterable<StatementContextBase<?, ?, ?>> effectiveChildrenToComplete();
+
+    // exposed for InferredStatementContext only
+    final void ensureCompletedPhase(final Mutable<?, ?, ?> stmt) {
+        verifyStatement(stmt);
+        final ModelProcessingPhase phase = completedPhase;
+        if (phase != null) {
+            ensureCompletedPhase((StatementContextBase<?, ?, ?>) stmt, phase);
         }
+    }
+
+    // Make sure target statement has transitioned at least to specified phase. This method is just before we take
+    // allow a statement to become our substatement. This is needed to ensure that every statement tree does not contain
+    // any statements which did not complete the same phase as the root statement.
+    private static void ensureCompletedPhase(final StatementContextBase<?, ?, ?> stmt,
+            final ModelProcessingPhase phase) {
+        verify(stmt.tryToCompletePhase(phase), "Statement %s cannot complete phase %s", stmt, phase);
+    }
+
+    private static void verifyStatement(final Mutable<?, ?, ?> stmt) {
+        verify(stmt instanceof StatementContextBase, "Unexpected statement %s", stmt);
+    }
 
-        statements.forEach(Objects::requireNonNull);
-        beforeAddEffectiveStatement(statements.size());
-        effective.addAll(statements);
+    private List<StatementContextBase<?, ?, ?>> beforeAddEffectiveStatement(
+            final List<StatementContextBase<?, ?, ?>> effective, final int toAdd) {
+        // We cannot allow statement to be further mutated
+        verify(completedPhase != ModelProcessingPhase.EFFECTIVE_MODEL, "Cannot modify finished statement at %s",
+            sourceReference());
+        return beforeAddEffectiveStatementUnsafe(effective, toAdd);
     }
 
-    private void beforeAddEffectiveStatement(final int toAdd) {
+    final List<StatementContextBase<?, ?, ?>> beforeAddEffectiveStatementUnsafe(
+            final List<StatementContextBase<?, ?, ?>> effective, final int toAdd) {
         final ModelProcessingPhase inProgressPhase = getRoot().getSourceContext().getInProgressPhase();
         checkState(inProgressPhase == ModelProcessingPhase.FULL_DECLARATION
                 || inProgressPhase == ModelProcessingPhase.EFFECTIVE_MODEL,
-                "Effective statement cannot be added in declared phase at: %s", getStatementSourceReference());
+                "Effective statement cannot be added in declared phase at: %s", sourceReference());
 
-        if (effective.isEmpty()) {
-            effective = new ArrayList<>(toAdd);
-        }
+        return effective.isEmpty() ? new ArrayList<>(toAdd) : effective;
     }
 
-    // Exists only due to memory optimization
+    // These two exists only due to memory optimization, should live in AbstractResumedStatement
     final boolean fullyDefined() {
         return fullyDefined;
     }
 
-    // Exists only due to memory optimization, should live in AbstractResumedStatement
     final void setFullyDefined() {
         fullyDefined = true;
     }
 
     @Override
     public E buildEffective() {
-        final E existing = effectiveInstance;
-        return existing != null ? existing : (effectiveInstance = definition.getFactory().createEffective(this));
+        final E existing;
+        return (existing = effectiveInstance) != null ? existing : loadEffective();
     }
 
+    private E loadEffective() {
+        // Creating an effective statement does not strictly require a declared instance -- there are statements like
+        // 'input', which are implicitly defined.
+        // Our implementation design makes an invariant assumption that buildDeclared() has been called by the time
+        // we attempt to create effective statement:
+        buildDeclared();
+
+        final E ret = effectiveInstance = createEffective();
+        // we have called createEffective(), substatements are no longer guarded by us. Let's see if we can clear up
+        // some residue.
+        releaseImplicitRef();
+        return ret;
+    }
+
+    // Exposed for ReplicaStatementContext
+    E createEffective() {
+        return definition.getFactory().createEffective(new BaseCurrentEffectiveStmtCtx<>(this), streamDeclared(),
+            streamEffective());
+    }
+
+    abstract Stream<? extends StmtContext<?, ?, ?>> streamDeclared();
+
+    abstract Stream<? extends StmtContext<?, ?, ?>> streamEffective();
+
     /**
-     * tries to execute current {@link ModelProcessingPhase} of source parsing.
+     * Try to execute current {@link ModelProcessingPhase} of source parsing. If the phase has already been executed,
+     * this method does nothing.
      *
-     * @param phase
-     *            to be executed (completed)
-     * @return if phase was successfully completed
-     * @throws SourceException
-     *             when an error occurred in source parsing
+     * @param phase to be executed (completed)
+     * @return true if phase was successfully completed
+     * @throws SourceException when an error occurred in source parsing
      */
-    boolean tryToCompletePhase(final ModelProcessingPhase phase) {
-
-        boolean finished = true;
-        final Collection<ContextMutation> openMutations = phaseMutation.get(phase);
-        if (!openMutations.isEmpty()) {
-            final Iterator<ContextMutation> it = openMutations.iterator();
-            while (it.hasNext()) {
-                final ContextMutation current = it.next();
-                if (current.isFinished()) {
-                    it.remove();
-                } else {
-                    finished = false;
-                }
-            }
+    final boolean tryToCompletePhase(final ModelProcessingPhase phase) {
+        return phase.isCompletedBy(completedPhase) || doTryToCompletePhase(phase);
+    }
 
-            if (openMutations.isEmpty()) {
-                phaseMutation.removeAll(phase);
-                if (phaseMutation.isEmpty()) {
-                    phaseMutation = ImmutableMultimap.of();
-                }
-            }
+    private boolean doTryToCompletePhase(final ModelProcessingPhase phase) {
+        final boolean finished = phaseMutation.isEmpty() ? true : runMutations(phase);
+        if (completeChildren(phase) && finished) {
+            onPhaseCompleted(phase);
+            return true;
         }
+        return false;
+    }
 
+    private boolean completeChildren(final ModelProcessingPhase phase) {
+        boolean finished = true;
         for (final StatementContextBase<?, ?, ?> child : mutableDeclaredSubstatements()) {
             finished &= child.tryToCompletePhase(phase);
         }
-        for (final Mutable<?, ?, ?> child : effective) {
-            if (child instanceof StatementContextBase) {
-                finished &= ((StatementContextBase<?, ?, ?>) child).tryToCompletePhase(phase);
+        for (final StatementContextBase<?, ?, ?> child : effectiveChildrenToComplete()) {
+            finished &= child.tryToCompletePhase(phase);
+        }
+        return finished;
+    }
+
+    private boolean runMutations(final ModelProcessingPhase phase) {
+        final Collection<ContextMutation> openMutations = phaseMutation.get(phase);
+        return openMutations.isEmpty() ? true : runMutations(phase, openMutations);
+    }
+
+    private boolean runMutations(final ModelProcessingPhase phase, final Collection<ContextMutation> openMutations) {
+        boolean finished = true;
+        final Iterator<ContextMutation> it = openMutations.iterator();
+        while (it.hasNext()) {
+            final ContextMutation current = it.next();
+            if (current.isFinished()) {
+                it.remove();
+            } else {
+                finished = false;
             }
         }
 
-        if (finished) {
-            onPhaseCompleted(phase);
-            return true;
+        if (openMutations.isEmpty()) {
+            phaseMutation.removeAll(phase);
+            cleanupPhaseMutation();
+        }
+        return finished;
+    }
+
+    private void cleanupPhaseMutation() {
+        if (phaseMutation.isEmpty()) {
+            phaseMutation = ImmutableMultimap.of();
         }
-        return false;
     }
 
     /**
@@ -564,10 +649,12 @@ public abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E
         completedPhase = phase;
 
         final Collection<OnPhaseFinished> listeners = phaseListeners.get(phase);
-        if (listeners.isEmpty()) {
-            return;
+        if (!listeners.isEmpty()) {
+            runPhaseListeners(phase, listeners);
         }
+    }
 
+    private void runPhaseListeners(final ModelProcessingPhase phase, final Collection<OnPhaseFinished> listeners) {
         final Iterator<OnPhaseFinished> listener = listeners.iterator();
         while (listener.hasNext()) {
             final OnPhaseFinished next = listener.next();
@@ -683,12 +770,12 @@ public abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E
     }
 
     @Override
-    public StatementDefinition getPublicDefinition() {
+    public final StatementDefinition publicDefinition() {
         return definition.getPublicView();
     }
 
     @Override
-    public ModelActionBuilder newInferenceAction(final ModelProcessingPhase phase) {
+    public final ModelActionBuilder newInferenceAction(final ModelProcessingPhase phase) {
         return getRoot().getSourceContext().newInferenceAction(phase);
     }
 
@@ -705,8 +792,8 @@ public abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E
      * @throws NullPointerException if any of the arguments is null
      */
     void addPhaseCompletedListener(final ModelProcessingPhase phase, final OnPhaseFinished listener) {
-        checkNotNull(phase, "Statement context processing phase cannot be null at: %s", getStatementSourceReference());
-        checkNotNull(listener, "Statement context phase listener cannot be null at: %s", getStatementSourceReference());
+        checkNotNull(phase, "Statement context processing phase cannot be null at: %s", sourceReference());
+        checkNotNull(listener, "Statement context phase listener cannot be null at: %s", sourceReference());
 
         ModelProcessingPhase finishedPhase = completedPhase;
         while (finishedPhase != null) {
@@ -726,14 +813,13 @@ public abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E
     /**
      * Adds a {@link ContextMutation} to a {@link ModelProcessingPhase}.
      *
-     * @throws IllegalStateException
-     *             when the mutation was registered after phase was completed
+     * @throws IllegalStateException when the mutation was registered after phase was completed
      */
-    void addMutation(final ModelProcessingPhase phase, final ContextMutation mutation) {
+    final void addMutation(final ModelProcessingPhase phase, final ContextMutation mutation) {
         ModelProcessingPhase finishedPhase = completedPhase;
         while (finishedPhase != null) {
             checkState(!phase.equals(finishedPhase), "Mutation registered after phase was completed at: %s",
-                getStatementSourceReference());
+                sourceReference());
             finishedPhase = finishedPhase.getPreviousPhase();
         }
 
@@ -743,8 +829,15 @@ public abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E
         phaseMutation.put(phase, mutation);
     }
 
+    final void removeMutation(final ModelProcessingPhase phase, final ContextMutation mutation) {
+        if (!phaseMutation.isEmpty()) {
+            phaseMutation.remove(phase, mutation);
+            cleanupPhaseMutation();
+        }
+    }
+
     @Override
-    public <K, KT extends K, N extends StatementNamespace<K, ?, ?>> void addContext(final Class<N> namespace,
+    public <K, KT extends K, N extends StatementNamespace<K, ?, ?>> void addContext(final Class<@NonNull N> namespace,
             final KT key,final StmtContext<?, ?, ?> stmt) {
         addContextToNamespace(namespace, key, stmt);
     }
@@ -753,7 +846,32 @@ public abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E
     public Optional<? extends Mutable<?, ?, ?>> copyAsChildOf(final Mutable<?, ?, ?> parent, final CopyType type,
             final QNameModule targetModule) {
         checkEffectiveModelCompleted(this);
-        return definition.support().copyAsChildOf(this, parent, type, targetModule);
+
+        final StatementSupport<A, D, E> support = definition.support();
+        final CopyPolicy policy = support.applyCopyPolicy(this, parent, type, targetModule);
+        switch (policy) {
+            case CONTEXT_INDEPENDENT:
+                if (hasEmptySubstatements()) {
+                    // This statement is context-independent and has no substatements -- hence it can be freely shared.
+                    return Optional.of(replicaAsChildOf(parent));
+                }
+                // FIXME: YANGTOOLS-694: filter out all context-independent substatements, eliminate fall-through
+                // fall through
+            case DECLARED_COPY:
+                // FIXME: YANGTOOLS-694: this is still to eager, we really want to copy as a lazily-instantiated
+                //                       context, so that we can support building an effective statement without copying
+                //                       anything -- we will typically end up not being inferred against. In that case,
+                //                       this slim context should end up dealing with differences at buildContext()
+                //                       time. This is a YANGTOOLS-1067 prerequisite (which will deal with what can and
+                //                       cannot be shared across instances).
+                return Optional.of(parent.childCopyOf(this, type, targetModule));
+            case IGNORE:
+                return Optional.empty();
+            case REJECT:
+                throw new IllegalStateException("Statement " + support.getPublicView() + " should never be copied");
+            default:
+                throw new IllegalStateException("Unhandled policy " + policy);
+        }
     }
 
     @Override
@@ -767,15 +885,15 @@ public abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E
     private <X, Y extends DeclaredStatement<X>, Z extends EffectiveStatement<X, Y>> Mutable<X, Y, Z> childCopyOf(
             final StatementContextBase<X, Y, Z> original, final CopyType type, final QNameModule targetModule) {
         final Optional<StatementSupport<?, ?, ?>> implicitParent = definition.getImplicitParentFor(
-            original.getPublicDefinition());
+            original.publicDefinition());
 
         final StatementContextBase<X, Y, Z> result;
         final InferredStatementContext<X, Y, Z> copy;
 
         if (implicitParent.isPresent()) {
             final StatementDefinitionContext<?, ?, ?> def = new StatementDefinitionContext<>(implicitParent.get());
-            result = new SubstatementContext(this, def, original.getStatementSourceReference(),
-                original.rawStatementArgument(), original.getStatementArgument(), type);
+            result = new SubstatementContext(this, def, original.sourceReference(), original.rawArgument(),
+                original.argument(), type);
 
             final CopyType childCopyType;
             switch (type) {
@@ -801,6 +919,16 @@ public abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E
         return result;
     }
 
+    @Override
+    public final StatementContextBase<A, D, E> replicaAsChildOf(final Mutable<?, ?, ?> parent) {
+        checkArgument(parent instanceof StatementContextBase, "Unsupported parent %s", parent);
+        return replicaAsChildOf((StatementContextBase<?, ?, ?>) parent);
+    }
+
+    final @NonNull StatementContextBase<A, D, E> replicaAsChildOf(final StatementContextBase<?, ?, ?> stmt) {
+        return new ReplicaStatementContext<>(stmt, this);
+    }
+
     private static void checkEffectiveModelCompleted(final StmtContext<?, ?, ?> stmt) {
         final ModelProcessingPhase phase = stmt.getCompletedPhase();
         checkState(phase == ModelProcessingPhase.EFFECTIVE_MODEL,
@@ -815,7 +943,7 @@ public abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E
     @Beta
     public final StatementContextBase<?, ?, ?> wrapWithImplicit(final StatementContextBase<?, ?, ?> original) {
         final Optional<StatementSupport<?, ?, ?>> optImplicit = definition.getImplicitParentFor(
-            original.getPublicDefinition());
+            original.publicDefinition());
         if (optImplicit.isEmpty()) {
             return original;
         }
@@ -823,8 +951,7 @@ public abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E
         final StatementDefinitionContext<?, ?, ?> def = new StatementDefinitionContext<>(optImplicit.get());
         final CopyType type = original.getCopyHistory().getLastOperation();
         final SubstatementContext<?, ?, ?> result = new SubstatementContext(original.getParentContext(), def,
-            original.getStatementSourceReference(), original.rawStatementArgument(), original.getStatementArgument(),
-            type);
+            original.sourceReference(), original.rawArgument(), original.argument(), type);
 
         result.addEffectiveSubstatement(original.reparent(result));
         result.setCompletedPhase(original.getCompletedPhase());
@@ -833,6 +960,18 @@ public abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E
 
     abstract StatementContextBase<A, D, E> reparent(StatementContextBase<?, ?, ?> newParent);
 
+    /**
+     * Indicate that the set of substatements is empty. This is a preferred shortcut to substatement stream filtering.
+     *
+     * @return True if {@link #allSubstatements()} and {@link #allSubstatementsStream()} would return an empty stream.
+     */
+    abstract boolean hasEmptySubstatements();
+
+    @Override
+    final boolean noImplictRef() {
+        return effectiveInstance != null || !isSupportedToBuildEffective();
+    }
+
     /**
      * Config statements are not all that common which means we are performing a recursive search towards the root
      * every time {@link #isConfiguration()} is invoked. This is quite expensive because it causes a linear search
@@ -856,14 +995,13 @@ public abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E
             return true;
         }
 
-        final StmtContext<Boolean, ?, ?> configStatement = StmtContextUtils.findFirstSubstatement(this,
-            ConfigStatement.class);
         final boolean isConfig;
-        if (configStatement != null) {
-            isConfig = configStatement.coerceStatementArgument();
+        final Optional<Boolean> optConfig = findSubstatementArgument(ConfigEffectiveStatement.class);
+        if (optConfig.isPresent()) {
+            isConfig = optConfig.orElseThrow();
             if (isConfig) {
                 // Validity check: if parent is config=false this cannot be a config=true
-                InferenceException.throwIf(!parent.isConfiguration(), getStatementSourceReference(),
+                InferenceException.throwIf(!parent.isConfiguration(), sourceReference(),
                         "Parent node has config=false, this node must not be specifed as config=true");
             }
         } else {
@@ -922,15 +1060,17 @@ public abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E
         return false;
     }
 
-    // Exists only to support SubstatementContext/InferredStatementContext
+    abstract @NonNull Optional<SchemaPath> schemaPath();
+
+    // Exists only to support {SubstatementContext,InferredStatementContext}.schemaPath()
+    @Deprecated
     final @NonNull Optional<SchemaPath> substatementGetSchemaPath() {
         SchemaPath local = schemaPath;
         if (local == null) {
             synchronized (this) {
                 local = schemaPath;
                 if (local == null) {
-                    local = createSchemaPath(coerceParentContext());
-                    schemaPath = local;
+                    schemaPath = local = createSchemaPath((StatementContextBase<?, ?, ?>) coerceParentContext());
                 }
             }
         }
@@ -938,18 +1078,19 @@ public abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E
         return Optional.ofNullable(local);
     }
 
-    private SchemaPath createSchemaPath(final Mutable<?, ?, ?> parent) {
-        final Optional<SchemaPath> maybeParentPath = parent.getSchemaPath();
+    @Deprecated
+    private SchemaPath createSchemaPath(final StatementContextBase<?, ?, ?> parent) {
+        final Optional<SchemaPath> maybeParentPath = parent.schemaPath();
         verify(maybeParentPath.isPresent(), "Parent %s does not have a SchemaPath", parent);
         final SchemaPath parentPath = maybeParentPath.get();
 
         if (StmtContextUtils.isUnknownStatement(this)) {
-            return parentPath.createChild(getPublicDefinition().getStatementName());
+            return parentPath.createChild(publicDefinition().getStatementName());
         }
-        final Object argument = getStatementArgument();
+        final Object argument = argument();
         if (argument instanceof QName) {
             final QName qname = (QName) argument;
-            if (StmtContextUtils.producesDeclared(this, UsesStatement.class)) {
+            if (producesDeclared(UsesStatement.class)) {
                 return maybeParentPath.orElse(null);
             }
 
@@ -962,11 +1103,10 @@ public abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E
             return parentPath.createChild(qname);
         }
         if (argument instanceof SchemaNodeIdentifier
-                && (StmtContextUtils.producesDeclared(this, AugmentStatement.class)
-                        || StmtContextUtils.producesDeclared(this, RefineStatement.class)
-                        || StmtContextUtils.producesDeclared(this, DeviationStatement.class))) {
+                && (producesDeclared(AugmentStatement.class) || producesDeclared(RefineStatement.class)
+                        || producesDeclared(DeviationStatement.class))) {
 
-            return parentPath.createChild(((SchemaNodeIdentifier) argument).getPathFromRoot());
+            return parentPath.createChild(((SchemaNodeIdentifier) argument).getNodeIdentifiers());
         }
 
         // FIXME: this does not look right
@@ -979,6 +1119,6 @@ public abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E
     }
 
     protected ToStringHelper addToStringAttributes(final ToStringHelper toStringHelper) {
-        return toStringHelper.add("definition", definition).add("rawArgument", rawStatementArgument());
+        return toStringHelper.add("definition", definition).add("rawArgument", rawArgument());
     }
 }