Use simple requireNonNull()
[yangtools.git] / yang / yang-parser-reactor / src / main / java / org / opendaylight / yangtools / yang / parser / stmt / reactor / StatementContextBase.java
index 6903620c32347e7c9adc0b61a3c7ae917a2aee53..a331d6b6d91619da119fcc3323aa94dee5a478a5 100644 (file)
@@ -8,7 +8,6 @@
 package org.opendaylight.yangtools.yang.parser.stmt.reactor;
 
 import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkNotNull;
 import static com.google.common.base.Preconditions.checkState;
 import static com.google.common.base.Verify.verify;
 import static java.util.Objects.requireNonNull;
@@ -34,15 +33,17 @@ import org.eclipse.jdt.annotation.Nullable;
 import org.opendaylight.yangtools.yang.common.QNameModule;
 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.parser.spi.meta.CopyHistory;
 import org.opendaylight.yangtools.yang.parser.spi.meta.CopyType;
 import org.opendaylight.yangtools.yang.parser.spi.meta.ImplicitParentAwareStatementSupport;
 import org.opendaylight.yangtools.yang.parser.spi.meta.ModelActionBuilder;
 import org.opendaylight.yangtools.yang.parser.spi.meta.ModelProcessingPhase;
+import org.opendaylight.yangtools.yang.parser.spi.meta.MutableStatement;
 import org.opendaylight.yangtools.yang.parser.spi.meta.NamespaceBehaviour;
 import org.opendaylight.yangtools.yang.parser.spi.meta.NamespaceKeyCriterion;
+import org.opendaylight.yangtools.yang.parser.spi.meta.ParserNamespace;
+import org.opendaylight.yangtools.yang.parser.spi.meta.StatementFactory;
 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;
@@ -128,14 +129,6 @@ public abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E
         return effectOfStatement;
     }
 
-    @Override
-    public void addAsEffectOfStatement(final StmtContext<?, ?, ?> ctx) {
-        if (effectOfStatement.isEmpty()) {
-            effectOfStatement = new ArrayList<>(1);
-        }
-        effectOfStatement.add(ctx);
-    }
-
     @Override
     public void addAsEffectOfStatement(final Collection<? extends StmtContext<?, ?, ?>> ctxs) {
         if (ctxs.isEmpty()) {
@@ -165,7 +158,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(
+    public final <K, V, T extends K, U extends V, N extends ParserNamespace<K, V>> void addToNs(
             final Class<@NonNull N> type, final T key, final U value) {
         addToNamespace(type, key, value);
     }
@@ -292,11 +285,11 @@ public abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E
     final List<ReactorStmtCtx<?, ?, ?>> addEffectiveSubstatementsImpl(final List<ReactorStmtCtx<?, ?, ?>> effective,
             final Collection<? extends Mutable<?, ?, ?>> statements) {
         final List<ReactorStmtCtx<?, ?, ?>> resized = beforeAddEffectiveStatement(effective, statements.size());
-        final Collection<? extends StatementContextBase<?, ?, ?>> casted =
-            (Collection<? extends StatementContextBase<?, ?, ?>>) statements;
+        final Collection<? extends ReactorStmtCtx<?, ?, ?>> casted =
+            (Collection<? extends ReactorStmtCtx<?, ?, ?>>) statements;
         final ModelProcessingPhase phase = completedPhase;
         if (phase != null) {
-            for (StatementContextBase<?, ?, ?> stmt : casted) {
+            for (ReactorStmtCtx<?, ?, ?> stmt : casted) {
                 ensureCompletedPhase(stmt, phase);
             }
         }
@@ -345,15 +338,42 @@ public abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E
         return effective.isEmpty() ? new ArrayList<>(toAdd) : effective;
     }
 
-    // Exposed for ReplicaStatementContext
     @Override
-    E createEffective() {
-        return definition.getFactory().createEffective(this, streamDeclared(), streamEffective());
+    final E createEffective() {
+        final E result = createEffective(definition.getFactory());
+        if (result instanceof MutableStatement) {
+            getRoot().addMutableStmtToSeal((MutableStatement) result);
+        }
+        return result;
     }
 
-    abstract Stream<? extends StmtContext<?, ?, ?>> streamDeclared();
+    @NonNull E createEffective(final StatementFactory<A, D, E> factory) {
+        return createEffective(factory, this);
+    }
+
+    // Creates EffectiveStatement through full materialization
+    static <A, D extends DeclaredStatement<A>, E extends EffectiveStatement<A, D>> @NonNull E createEffective(
+            final StatementFactory<A, D, E> factory, final StatementContextBase<A, D, E> ctx) {
+        return factory.createEffective(ctx, ctx.streamDeclared(), ctx.streamEffective());
+    }
 
-    abstract Stream<? extends StmtContext<?, ?, ?>> streamEffective();
+    /**
+     * Return a stream of declared statements which can be built into an {@link EffectiveStatement}, as per
+     * {@link StmtContext#buildEffective()} contract.
+     *
+     * @return Stream of supported declared statements.
+     */
+    // FIXME: we really want to unify this with streamEffective(), under its name
+    abstract Stream<? extends @NonNull StmtContext<?, ?, ?>> streamDeclared();
+
+    /**
+     * Return a stream of inferred statements which can be built into an {@link EffectiveStatement}, as per
+     * {@link StmtContext#buildEffective()} contract.
+     *
+     * @return Stream of supported effective statements.
+     */
+    // FIXME: this method is currently a misnomer, but unifying with streamDeclared() would make this accurate again
+    abstract Stream<? extends @NonNull StmtContext<?, ?, ?>> streamEffective();
 
     @Override
     final boolean doTryToCompletePhase(final ModelProcessingPhase phase) {
@@ -416,6 +436,9 @@ public abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E
      */
     private void onPhaseCompleted(final ModelProcessingPhase phase) {
         completedPhase = phase;
+        if (phase == ModelProcessingPhase.EFFECTIVE_MODEL) {
+            summarizeSubstatementPolicy();
+        }
 
         final Collection<OnPhaseFinished> listeners = phaseListeners.get(phase);
         if (!listeners.isEmpty()) {
@@ -423,6 +446,53 @@ public abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E
         }
     }
 
+    private void summarizeSubstatementPolicy() {
+        if (definition().support().copyPolicy() == CopyPolicy.EXACT_REPLICA || noSensitiveSubstatements()) {
+            setAllSubstatementsContextIndependent();
+        }
+    }
+
+    /**
+     * Determine whether any substatements are copy-sensitive as determined by {@link StatementSupport#copyPolicy()}.
+     * Only {@link CopyPolicy#CONTEXT_INDEPENDENT}, {@link CopyPolicy#EXACT_REPLICA} and {@link CopyPolicy#IGNORE} are
+     * copy-insensitive. Note that statements which are not {@link StmtContext#isSupportedToBuildEffective()} are all
+     * considered copy-insensitive.
+     *
+     * <p>
+     * Implementations are expected to call {@link #noSensitiveSubstatements()} to actually traverse substatement sets.
+     *
+     * @return True if no substatements require copy-sensitive handling
+     */
+    abstract boolean noSensitiveSubstatements();
+
+    /**
+     * Determine whether any of the provided substatements are context-sensitive for purposes of implementing
+     * {@link #noSensitiveSubstatements()}.
+     *
+     * @param substatements Substatements to check
+     * @return True if no substatements require context-sensitive handling
+     */
+    static boolean noSensitiveSubstatements(final Collection<? extends ReactorStmtCtx<?, ?, ?>> substatements) {
+        for (ReactorStmtCtx<?, ?, ?> stmt : substatements) {
+            if (stmt.isSupportedToBuildEffective()) {
+                if (!stmt.allSubstatementsContextIndependent()) {
+                    // This is a recursive property
+                    return false;
+                }
+
+                switch (stmt.definition().support().copyPolicy()) {
+                    case CONTEXT_INDEPENDENT:
+                    case EXACT_REPLICA:
+                    case IGNORE:
+                        break;
+                    default:
+                        return false;
+                }
+            }
+        }
+        return true;
+    }
+
     private void runPhaseListeners(final ModelProcessingPhase phase, final Collection<OnPhaseFinished> listeners) {
         final Iterator<OnPhaseFinished> listener = listeners.iterator();
         while (listener.hasNext()) {
@@ -452,7 +522,7 @@ public abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E
         return definition;
     }
 
-    final <K, V, N extends IdentifierNamespace<K, V>> void onNamespaceItemAddedAction(final Class<N> type, final K key,
+    final <K, V, N extends ParserNamespace<K, V>> void onNamespaceItemAddedAction(final Class<N> type, final K key,
             final OnNamespaceItemAdded listener) {
         final Object potential = getFromNamespace(type, key);
         if (potential != null) {
@@ -469,7 +539,7 @@ public abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E
         });
     }
 
-    final <K, V, N extends IdentifierNamespace<K, V>> void onNamespaceItemAddedAction(final Class<N> type,
+    final <K, V, N extends ParserNamespace<K, V>> void onNamespaceItemAddedAction(final Class<N> type,
             final ModelProcessingPhase phase, final NamespaceKeyCriterion<K> criterion,
             final OnNamespaceItemAdded listener) {
         final Optional<Entry<K, V>> existing = getFromNamespace(type, criterion);
@@ -495,7 +565,7 @@ public abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E
         });
     }
 
-    final <K, V, N extends IdentifierNamespace<K, V>> void selectMatch(final Class<N> type,
+    final <K, V, N extends ParserNamespace<K, V>> void selectMatch(final Class<N> type,
             final NamespaceKeyCriterion<K> criterion, final OnNamespaceItemAdded listener) {
         final Optional<Entry<K, V>> optMatch = getFromNamespace(type, criterion);
         checkState(optMatch.isPresent(), "Failed to find a match for criterion %s in namespace %s node %s", criterion,
@@ -504,7 +574,7 @@ public abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E
         listener.namespaceItemAdded(StatementContextBase.this, type, match.getKey(), match.getValue());
     }
 
-    final <K, V, N extends IdentifierNamespace<K, V>> void waitForPhase(final Object value, final Class<N> type,
+    final <K, V, N extends ParserNamespace<K, V>> void waitForPhase(final Object value, final Class<N> type,
             final ModelProcessingPhase phase, final NamespaceKeyCriterion<K> criterion,
             final OnNamespaceItemAdded listener) {
         ((StatementContextBase<?, ? ,?>) value).addPhaseCompletedListener(phase,
@@ -514,7 +584,7 @@ public abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E
             });
     }
 
-    private <K, V, N extends IdentifierNamespace<K, V>> NamespaceBehaviourWithListeners<K, V, N> getBehaviour(
+    private <K, V, N extends ParserNamespace<K, V>> NamespaceBehaviourWithListeners<K, V, N> getBehaviour(
             final Class<N> type) {
         final NamespaceBehaviour<K, V, N> behaviour = getBehaviourRegistry().getNamespaceBehaviour(type);
         checkArgument(behaviour instanceof NamespaceBehaviourWithListeners, "Namespace %s does not support listeners",
@@ -536,8 +606,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", sourceReference());
-        checkNotNull(listener, "Statement context phase listener cannot be null at: %s", sourceReference());
+        requireNonNull(phase, "Statement context processing phase cannot be null");
+        requireNonNull(listener, "Statement context phase listener cannot be null");
 
         ModelProcessingPhase finishedPhase = completedPhase;
         while (finishedPhase != null) {
@@ -590,27 +660,27 @@ 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 Optional.ofNullable(copyAsChildOfImpl(parent, type, targetModule));
+    }
 
+    private ReactorStmtCtx<A, D, E> copyAsChildOfImpl(final Mutable<?, ?, ?> parent, final CopyType type,
+            final QNameModule targetModule) {
         final StatementSupport<A, D, E> support = definition.support();
-        final CopyPolicy policy = support.applyCopyPolicy(this, parent, type, targetModule);
+        final CopyPolicy policy = support.copyPolicy();
         switch (policy) {
+            case EXACT_REPLICA:
+                return replicaAsChildOf(parent);
             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));
+                if (allSubstatementsContextIndependent()) {
+                    return 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));
+                // FIXME: ugly cast
+                return (ReactorStmtCtx<A, D, E>) parent.childCopyOf(this, type, targetModule);
             case IGNORE:
-                return Optional.empty();
+                return null;
             case REJECT:
                 throw new IllegalStateException("Statement " + support.getPublicView() + " should never be copied");
             default:
@@ -618,6 +688,26 @@ public abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E
         }
     }
 
+    @Override
+    final ReactorStmtCtx<?, ?, ?> asEffectiveChildOf(final StatementContextBase<?, ?, ?> parent, final CopyType type,
+            final QNameModule targetModule) {
+        final ReactorStmtCtx<A, D, E> copy = copyAsChildOfImpl(parent, type, targetModule);
+        if (copy == null) {
+            // The statement fizzled, this should never happen, perhaps a verify()?
+            return null;
+        }
+
+        parent.ensureCompletedPhase(copy);
+        return canReuseCurrent(copy) ? this : copy;
+    }
+
+    private boolean canReuseCurrent(final ReactorStmtCtx<A, D, E> copy) {
+        // Defer to statement factory to see if we can reuse this object. If we can and have only context-independent
+        // substatements we can reuse the object. More complex cases are handled indirectly via the copy.
+        return definition.getFactory().canReuseCurrent(copy, this, buildEffective().effectiveSubstatements())
+            && allSubstatementsContextIndependent();
+    }
+
     @Override
     public final Mutable<?, ?, ?> childCopyOf(final StmtContext<?, ?, ?> stmt, final CopyType type,
             final QNameModule targetModule) {
@@ -664,13 +754,8 @@ public abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E
     }
 
     @Override
-    public final ReactorStmtCtx<A, D, E> replicaAsChildOf(final Mutable<?, ?, ?> parent) {
-        checkArgument(parent instanceof StatementContextBase, "Unsupported parent %s", parent);
-        return replicaAsChildOf((StatementContextBase<?, ?, ?>) parent);
-    }
-
-    final @NonNull ReplicaStatementContext<A, D, E> replicaAsChildOf(final StatementContextBase<?, ?, ?> stmt) {
-        return new ReplicaStatementContext<>(stmt, this);
+    final ReplicaStatementContext<A, D, E> replicaAsChildOf(final StatementContextBase<?, ?, ?> parent) {
+        return new ReplicaStatementContext<>(parent, this);
     }
 
     private static void checkEffectiveModelCompleted(final StmtContext<?, ?, ?> stmt) {