Return ReactorStmtCtx from ensureCompletedPhase()
[yangtools.git] / parser / yang-parser-reactor / src / main / java / org / opendaylight / yangtools / yang / parser / stmt / reactor / StatementContextBase.java
1 /*
2  * Copyright (c) 2015 Cisco Systems, Inc. and others.  All rights reserved.
3  *
4  * This program and the accompanying materials are made available under the
5  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
6  * and is available at http://www.eclipse.org/legal/epl-v10.html
7  */
8 package org.opendaylight.yangtools.yang.parser.stmt.reactor;
9
10 import static com.google.common.base.Preconditions.checkArgument;
11 import static com.google.common.base.Preconditions.checkState;
12 import static com.google.common.base.Verify.verify;
13 import static com.google.common.base.Verify.verifyNotNull;
14 import static java.util.Objects.requireNonNull;
15
16 import com.google.common.collect.ImmutableCollection;
17 import com.google.common.collect.ImmutableList;
18 import com.google.common.collect.ImmutableMultimap;
19 import com.google.common.collect.Multimap;
20 import com.google.common.collect.Multimaps;
21 import java.util.ArrayList;
22 import java.util.Collection;
23 import java.util.Collections;
24 import java.util.EnumMap;
25 import java.util.EventListener;
26 import java.util.Iterator;
27 import java.util.List;
28 import java.util.Map.Entry;
29 import java.util.Optional;
30 import java.util.stream.Stream;
31 import org.eclipse.jdt.annotation.NonNull;
32 import org.eclipse.jdt.annotation.Nullable;
33 import org.opendaylight.yangtools.yang.common.QNameModule;
34 import org.opendaylight.yangtools.yang.model.api.meta.DeclaredStatement;
35 import org.opendaylight.yangtools.yang.model.api.meta.EffectiveStatement;
36 import org.opendaylight.yangtools.yang.model.api.meta.StatementDefinition;
37 import org.opendaylight.yangtools.yang.parser.spi.meta.CopyHistory;
38 import org.opendaylight.yangtools.yang.parser.spi.meta.CopyType;
39 import org.opendaylight.yangtools.yang.parser.spi.meta.ImplicitParentAwareStatementSupport;
40 import org.opendaylight.yangtools.yang.parser.spi.meta.ModelActionBuilder;
41 import org.opendaylight.yangtools.yang.parser.spi.meta.ModelProcessingPhase;
42 import org.opendaylight.yangtools.yang.parser.spi.meta.ModelProcessingPhase.ExecutionOrder;
43 import org.opendaylight.yangtools.yang.parser.spi.meta.MutableStatement;
44 import org.opendaylight.yangtools.yang.parser.spi.meta.NamespaceBehaviour;
45 import org.opendaylight.yangtools.yang.parser.spi.meta.NamespaceKeyCriterion;
46 import org.opendaylight.yangtools.yang.parser.spi.meta.ParserNamespace;
47 import org.opendaylight.yangtools.yang.parser.spi.meta.StatementFactory;
48 import org.opendaylight.yangtools.yang.parser.spi.meta.StatementNamespace;
49 import org.opendaylight.yangtools.yang.parser.spi.meta.StatementSupport;
50 import org.opendaylight.yangtools.yang.parser.spi.meta.StatementSupport.CopyPolicy;
51 import org.opendaylight.yangtools.yang.parser.spi.meta.StmtContext;
52 import org.opendaylight.yangtools.yang.parser.spi.meta.UndeclaredStatementFactory;
53 import org.opendaylight.yangtools.yang.parser.spi.source.SourceException;
54 import org.opendaylight.yangtools.yang.parser.stmt.reactor.NamespaceBehaviourWithListeners.KeyedValueAddedListener;
55 import org.opendaylight.yangtools.yang.parser.stmt.reactor.NamespaceBehaviourWithListeners.PredicateValueAddedListener;
56 import org.slf4j.Logger;
57 import org.slf4j.LoggerFactory;
58
59 /**
60  * Core reactor statement implementation of {@link Mutable}.
61  *
62  * @param <A> Argument type
63  * @param <D> Declared Statement representation
64  * @param <E> Effective Statement representation
65  */
66 abstract class StatementContextBase<A, D extends DeclaredStatement<A>, E extends EffectiveStatement<A, D>>
67         extends ReactorStmtCtx<A, D, E> implements CopyHistory {
68     /**
69      * Event listener when an item is added to model namespace.
70      */
71     interface OnNamespaceItemAdded extends EventListener {
72         /**
73          * Invoked whenever a new item is added to a namespace.
74          */
75         void namespaceItemAdded(StatementContextBase<?, ?, ?> context, Class<?> namespace, Object key, Object value);
76     }
77
78     /**
79      * Event listener when a parsing {@link ModelProcessingPhase} is completed.
80      */
81     interface OnPhaseFinished extends EventListener {
82         /**
83          * Invoked whenever a processing phase has finished.
84          */
85         boolean phaseFinished(StatementContextBase<?, ?, ?> context, ModelProcessingPhase finishedPhase);
86     }
87
88     /**
89      * Interface for all mutations within an {@link ModelActionBuilder.InferenceAction}.
90      */
91     interface ContextMutation {
92
93         boolean isFinished();
94     }
95
96     private static final Logger LOG = LoggerFactory.getLogger(StatementContextBase.class);
97
98     // Bottom 4 bits, encoding a CopyHistory, aight?
99     private static final byte COPY_ORIGINAL              = 0x00;
100     private static final byte COPY_LAST_TYPE_MASK        = 0x03;
101     @Deprecated(since = "7.0.9", forRemoval = true)
102     private static final byte COPY_ADDED_BY_USES         = 0x04;
103     private static final byte COPY_ADDED_BY_AUGMENTATION = 0x08;
104
105     // Top four bits, of which we define the topmost two to 0. We use the bottom two to encode last CopyType, aight?
106     private static final int COPY_CHILD_TYPE_SHIFT       = 4;
107
108     private static final CopyType @NonNull [] COPY_TYPE_VALUES = CopyType.values();
109
110     static {
111         final int copyTypes = COPY_TYPE_VALUES.length;
112         // This implies CopyType.ordinal() is <= COPY_TYPE_MASK
113         verify(copyTypes == COPY_LAST_TYPE_MASK + 1, "Unexpected %s CopyType values", copyTypes);
114     }
115
116     /**
117      * 8 bits worth of instance storage. This is treated as a constant bit field with following structure:
118      * <pre>
119      *   <code>
120      * |7|6|5|4|3|2|1|0|
121      * |0 0|cct|a|u|lst|
122      *   </code>
123      * </pre>
124      *
125      * <p>
126      * The four allocated fields are:
127      * <ul>
128      *   <li>{@code lst}, encoding the four states corresponding to {@link CopyHistory#getLastOperation()}</li>
129      *   <li>{@code u}, encoding {@link #isAddedByUses()}</li>
130      *   <li>{@code a}, encoding {@link #isAugmenting()}</li>
131      *   <li>{@code cct} encoding {@link #childCopyType()}</li>
132      * </ul>
133      * We still have two unused bits.
134      */
135     private final byte bitsAight;
136
137     // Note: this field can strictly be derived in InferredStatementContext, but it forms the basis of many of our
138     //       operations, hence we want to keep it close by.
139     private final @NonNull StatementDefinitionContext<A, D, E> definition;
140
141     // TODO: consider keying by Byte equivalent of ExecutionOrder
142     private Multimap<ModelProcessingPhase, OnPhaseFinished> phaseListeners = ImmutableMultimap.of();
143     private Multimap<ModelProcessingPhase, ContextMutation> phaseMutation = ImmutableMultimap.of();
144
145     private List<StmtContext<?, ?, ?>> effectOfStatement = ImmutableList.of();
146
147     /**
148      * {@link ModelProcessingPhase.ExecutionOrder} value of current {@link ModelProcessingPhase} of this statement.
149      */
150     private byte executionOrder;
151
152     // TODO: we a single byte of alignment shadow left, we should think how we can use it to cache information we build
153     //       during InferredStatementContext.tryToReusePrototype(). We usually end up being routed to
154     //       copyAsChildOfImpl() -- which performs an eager instantiation and checks for changes afterwards. We should
155     //       be able to capture how parent scope affects the copy in a few bits. If we can do that, than we can reap
156     //       the benefits by just examining new parent context and old parent context contribution to the state. If
157     //       their impact is the same, we can skip instantiation of statements and directly reuse them (individually,
158     //       or as a complete file).
159     //
160     //       Whatever we end up tracking, we need to track two views of that -- for the statement itself
161     //       (sans substatements) and a summary of substatements. I think it should be possible to get this working
162     //       with 2x5bits -- we have up to 15 mutable bits available if we share the field with implicitDeclaredFlag.
163
164     // Copy constructor used by subclasses to implement reparent()
165     StatementContextBase(final StatementContextBase<A, D, E> original) {
166         super(original);
167         this.bitsAight = original.bitsAight;
168         this.definition = original.definition;
169         this.executionOrder = original.executionOrder;
170     }
171
172     StatementContextBase(final StatementDefinitionContext<A, D, E> def) {
173         this.definition = requireNonNull(def);
174         this.bitsAight = COPY_ORIGINAL;
175     }
176
177     StatementContextBase(final StatementDefinitionContext<A, D, E> def, final CopyType copyType) {
178         this.definition = requireNonNull(def);
179         this.bitsAight = (byte) copyFlags(copyType);
180     }
181
182     StatementContextBase(final StatementContextBase<A, D, E> prototype, final CopyType copyType,
183             final CopyType childCopyType) {
184         this.definition = prototype.definition;
185         this.bitsAight = (byte) (copyFlags(copyType)
186             | prototype.bitsAight & ~COPY_LAST_TYPE_MASK | childCopyType.ordinal() << COPY_CHILD_TYPE_SHIFT);
187     }
188
189     private static int copyFlags(final CopyType copyType) {
190         return historyFlags(copyType) | copyType.ordinal();
191     }
192
193     private static byte historyFlags(final CopyType copyType) {
194         return switch (copyType) {
195             case ADDED_BY_AUGMENTATION -> COPY_ADDED_BY_AUGMENTATION;
196             case ADDED_BY_USES -> COPY_ADDED_BY_USES;
197             case ADDED_BY_USES_AUGMENTATION -> COPY_ADDED_BY_AUGMENTATION | COPY_ADDED_BY_USES;
198             case ORIGINAL -> COPY_ORIGINAL;
199         };
200     }
201
202     @Override
203     public final Collection<? extends StmtContext<?, ?, ?>> getEffectOfStatement() {
204         return effectOfStatement;
205     }
206
207     @Override
208     public final void addAsEffectOfStatement(final Collection<? extends StmtContext<?, ?, ?>> ctxs) {
209         if (ctxs.isEmpty()) {
210             return;
211         }
212
213         if (effectOfStatement.isEmpty()) {
214             effectOfStatement = new ArrayList<>(ctxs.size());
215         }
216         effectOfStatement.addAll(ctxs);
217     }
218
219     //
220     // CopyHistory integration
221     //
222
223     @Override
224     public final CopyHistory history() {
225         return this;
226     }
227
228     @Override
229     @Deprecated(since = "7.0.9", forRemoval = true)
230     public final boolean isAddedByUses() {
231         return (bitsAight & COPY_ADDED_BY_USES) != 0;
232     }
233
234     @Override
235     @Deprecated(since = "8.0.0")
236     public final boolean isAugmenting() {
237         return (bitsAight & COPY_ADDED_BY_AUGMENTATION) != 0;
238     }
239
240     @Override
241     public final CopyType getLastOperation() {
242         return COPY_TYPE_VALUES[bitsAight & COPY_LAST_TYPE_MASK];
243     }
244
245     // This method exists only for space optimization of InferredStatementContext
246     final CopyType childCopyType() {
247         return COPY_TYPE_VALUES[bitsAight >> COPY_CHILD_TYPE_SHIFT & COPY_LAST_TYPE_MASK];
248     }
249
250     //
251     // Inference completion tracking
252     //
253
254     @Override
255     final byte executionOrder() {
256         return executionOrder;
257     }
258
259     // FIXME: this should be propagated through a correct constructor
260     @Deprecated
261     final void setCompletedPhase(final ModelProcessingPhase completedPhase) {
262         this.executionOrder = completedPhase.executionOrder();
263     }
264
265     @Override
266     public final <K, V, T extends K, U extends V, N extends ParserNamespace<K, V>> void addToNs(
267             final Class<@NonNull N> type, final T key, final U value) {
268         addToNamespace(type, key, value);
269     }
270
271     static final Collection<? extends Mutable<?, ?, ?>> mutableEffectiveSubstatements(
272             final List<ReactorStmtCtx<?, ?, ?>> effective) {
273         return effective instanceof ImmutableCollection ? effective : Collections.unmodifiableCollection(effective);
274     }
275
276     private static List<ReactorStmtCtx<?, ?, ?>> shrinkEffective(final List<ReactorStmtCtx<?, ?, ?>> effective) {
277         return effective.isEmpty() ? ImmutableList.of() : effective;
278     }
279
280     static final List<ReactorStmtCtx<?, ?, ?>> removeStatementFromEffectiveSubstatements(
281             final List<ReactorStmtCtx<?, ?, ?>> effective, final StatementDefinition statementDef) {
282         if (effective.isEmpty()) {
283             return effective;
284         }
285
286         final Iterator<? extends StmtContext<?, ?, ?>> iterator = effective.iterator();
287         while (iterator.hasNext()) {
288             final StmtContext<?, ?, ?> next = iterator.next();
289             if (statementDef.equals(next.publicDefinition())) {
290                 iterator.remove();
291             }
292         }
293
294         return shrinkEffective(effective);
295     }
296
297     static final List<ReactorStmtCtx<?, ?, ?>> removeStatementFromEffectiveSubstatements(
298             final List<ReactorStmtCtx<?, ?, ?>> effective, final StatementDefinition statementDef,
299             final String statementArg) {
300         if (statementArg == null) {
301             return removeStatementFromEffectiveSubstatements(effective, statementDef);
302         }
303
304         if (effective.isEmpty()) {
305             return effective;
306         }
307
308         final Iterator<ReactorStmtCtx<?, ?, ?>> iterator = effective.iterator();
309         while (iterator.hasNext()) {
310             final Mutable<?, ?, ?> next = iterator.next();
311             if (statementDef.equals(next.publicDefinition()) && statementArg.equals(next.rawArgument())) {
312                 iterator.remove();
313             }
314         }
315
316         return shrinkEffective(effective);
317     }
318
319     @Override
320     public final <X, Y extends DeclaredStatement<X>, Z extends EffectiveStatement<X, Y>>
321             Mutable<X, Y, Z> createUndeclaredSubstatement(final StatementSupport<X, Y, Z> support, final X arg) {
322         requireNonNull(support);
323         checkArgument(support instanceof UndeclaredStatementFactory, "Unsupported statement support %s", support);
324
325         final var ret = new UndeclaredStmtCtx<>(this, support, arg);
326         support.onStatementAdded(ret);
327         return ret;
328     }
329
330     final List<ReactorStmtCtx<?, ?, ?>> addEffectiveSubstatement(final List<ReactorStmtCtx<?, ?, ?>> effective,
331             final Mutable<?, ?, ?> substatement) {
332         final ReactorStmtCtx<?, ?, ?> stmt = verifyStatement(substatement);
333         final List<ReactorStmtCtx<?, ?, ?>> resized = beforeAddEffectiveStatement(effective, 1);
334         ensureCompletedExecution(stmt);
335         resized.add(stmt);
336         return resized;
337     }
338
339     static final void afterAddEffectiveSubstatement(final Mutable<?, ?, ?> substatement) {
340         // Undeclared statements still need to have 'onDeclarationFinished()' triggered
341         if (substatement instanceof UndeclaredStmtCtx) {
342             finishDeclaration((UndeclaredStmtCtx<?, ?, ?>) substatement);
343         }
344     }
345
346     // Split out to keep generics working without a warning
347     private static <X, Y extends DeclaredStatement<X>, Z extends EffectiveStatement<X, Y>> void finishDeclaration(
348             final UndeclaredStmtCtx<X, Y, Z> substatement) {
349         substatement.definition().onDeclarationFinished(substatement, ModelProcessingPhase.FULL_DECLARATION);
350     }
351
352     @Override
353     public final void addEffectiveSubstatements(final Collection<? extends Mutable<?, ?, ?>> statements) {
354         if (!statements.isEmpty()) {
355             statements.forEach(StatementContextBase::verifyStatement);
356             addEffectiveSubstatementsImpl(statements);
357         }
358     }
359
360     abstract void addEffectiveSubstatementsImpl(Collection<? extends Mutable<?, ?, ?>> statements);
361
362     final List<ReactorStmtCtx<?, ?, ?>> addEffectiveSubstatementsImpl(final List<ReactorStmtCtx<?, ?, ?>> effective,
363             final Collection<? extends Mutable<?, ?, ?>> statements) {
364         final List<ReactorStmtCtx<?, ?, ?>> resized = beforeAddEffectiveStatement(effective, statements.size());
365         final Collection<? extends ReactorStmtCtx<?, ?, ?>> casted =
366             (Collection<? extends ReactorStmtCtx<?, ?, ?>>) statements;
367         if (executionOrder != ExecutionOrder.NULL) {
368             for (ReactorStmtCtx<?, ?, ?> stmt : casted) {
369                 ensureCompletedExecution(stmt, executionOrder);
370             }
371         }
372
373         resized.addAll(casted);
374         return resized;
375     }
376
377     abstract Iterator<ReactorStmtCtx<?, ?, ?>> effectiveChildrenToComplete();
378
379     // exposed for InferredStatementContext only
380     final ReactorStmtCtx<?, ?, ?> ensureCompletedPhase(final Mutable<?, ?, ?> stmt) {
381         final var ret = verifyStatement(stmt);
382         ensureCompletedExecution(ret);
383         return ret;
384     }
385
386     // Make sure target statement has transitioned at least to our phase (if we have one). This method is just before we
387     // take allow a statement to become our substatement. This is needed to ensure that every statement tree does not
388     // contain any statements which did not complete the same phase as the root statement.
389     private void ensureCompletedExecution(final ReactorStmtCtx<?, ?, ?> stmt) {
390         if (executionOrder != ExecutionOrder.NULL) {
391             ensureCompletedExecution(stmt, executionOrder);
392         }
393     }
394
395     private static void ensureCompletedExecution(final ReactorStmtCtx<?, ?, ?> stmt, final byte executionOrder) {
396         verify(stmt.tryToCompletePhase(executionOrder), "Statement %s cannot complete phase %s", stmt, executionOrder);
397     }
398
399     private static ReactorStmtCtx<?, ?, ?> verifyStatement(final Mutable<?, ?, ?> stmt) {
400         verify(stmt instanceof ReactorStmtCtx, "Unexpected statement %s", stmt);
401         return (ReactorStmtCtx<?, ?, ?>) stmt;
402     }
403
404     private List<ReactorStmtCtx<?, ?, ?>> beforeAddEffectiveStatement(final List<ReactorStmtCtx<?, ?, ?>> effective,
405             final int toAdd) {
406         // We cannot allow statement to be further mutated.
407         // TODO: we really want to say 'not NULL and not at or after EFFECTIVE_MODEL here. This will matter if we have
408         //       a phase after EFFECTIVE_MODEL
409         verify(executionOrder != ExecutionOrder.EFFECTIVE_MODEL, "Cannot modify finished statement at %s",
410             sourceReference());
411         return beforeAddEffectiveStatementUnsafe(effective, toAdd);
412     }
413
414     final List<ReactorStmtCtx<?, ?, ?>> beforeAddEffectiveStatementUnsafe(final List<ReactorStmtCtx<?, ?, ?>> effective,
415             final int toAdd) {
416         final ModelProcessingPhase inProgressPhase = getRoot().getSourceContext().getInProgressPhase();
417         checkState(inProgressPhase == ModelProcessingPhase.FULL_DECLARATION
418                 || inProgressPhase == ModelProcessingPhase.EFFECTIVE_MODEL,
419                 "Effective statement cannot be added in declared phase at: %s", sourceReference());
420
421         return effective.isEmpty() ? new ArrayList<>(toAdd) : effective;
422     }
423
424     @Override
425     final E createEffective() {
426         final E result = createEffective(definition.getFactory());
427         if (result instanceof MutableStatement) {
428             getRoot().addMutableStmtToSeal((MutableStatement) result);
429         }
430         return result;
431     }
432
433     abstract @NonNull E createEffective(@NonNull StatementFactory<A, D, E> factory);
434
435     /**
436      * Return a stream of declared statements which can be built into an {@link EffectiveStatement}, as per
437      * {@link StmtContext#buildEffective()} contract.
438      *
439      * @return Stream of supported declared statements.
440      */
441     // FIXME: we really want to unify this with streamEffective(), under its name
442     abstract Stream<? extends @NonNull StmtContext<?, ?, ?>> streamDeclared();
443
444     /**
445      * Return a stream of inferred statements which can be built into an {@link EffectiveStatement}, as per
446      * {@link StmtContext#buildEffective()} contract.
447      *
448      * @return Stream of supported effective statements.
449      */
450     // FIXME: this method is currently a misnomer, but unifying with streamDeclared() would make this accurate again
451     abstract Stream<? extends @NonNull StmtContext<?, ?, ?>> streamEffective();
452
453     @Override
454     final boolean doTryToCompletePhase(final byte targetOrder) {
455         final boolean finished = phaseMutation.isEmpty() || runMutations(targetOrder);
456         if (completeChildren(targetOrder) && finished) {
457             onPhaseCompleted(targetOrder);
458             return true;
459         }
460         return false;
461     }
462
463     private boolean completeChildren(final byte targetOrder) {
464         boolean finished = true;
465         for (final StatementContextBase<?, ?, ?> child : mutableDeclaredSubstatements()) {
466             finished &= child.tryToCompletePhase(targetOrder);
467         }
468         final var it = effectiveChildrenToComplete();
469         while (it.hasNext()) {
470             finished &= it.next().tryToCompletePhase(targetOrder);
471         }
472         return finished;
473     }
474
475     private boolean runMutations(final byte targetOrder) {
476         final ModelProcessingPhase phase = verifyNotNull(ModelProcessingPhase.ofExecutionOrder(targetOrder));
477         final Collection<ContextMutation> openMutations = phaseMutation.get(phase);
478         return openMutations.isEmpty() || runMutations(phase, openMutations);
479     }
480
481     private boolean runMutations(final ModelProcessingPhase phase, final Collection<ContextMutation> openMutations) {
482         boolean finished = true;
483         final Iterator<ContextMutation> it = openMutations.iterator();
484         while (it.hasNext()) {
485             final ContextMutation current = it.next();
486             if (current.isFinished()) {
487                 it.remove();
488             } else {
489                 finished = false;
490             }
491         }
492
493         if (openMutations.isEmpty()) {
494             phaseMutation.removeAll(phase);
495             cleanupPhaseMutation();
496         }
497         return finished;
498     }
499
500     private void cleanupPhaseMutation() {
501         if (phaseMutation.isEmpty()) {
502             phaseMutation = ImmutableMultimap.of();
503         }
504     }
505
506     /**
507      * Occurs on end of {@link ModelProcessingPhase} of source parsing. This method must not be called with
508      * {@code executionOrder} equal to {@link ExecutionOrder#NULL}.
509      *
510      * @param phase that was to be completed (finished)
511      * @throws SourceException when an error occurred in source parsing
512      */
513     private void onPhaseCompleted(final byte completedOrder) {
514         executionOrder = completedOrder;
515         if (completedOrder == ExecutionOrder.EFFECTIVE_MODEL) {
516             // We have completed effective model, substatements are guaranteed not to change
517             summarizeSubstatementPolicy();
518         }
519
520         final ModelProcessingPhase phase = verifyNotNull(ModelProcessingPhase.ofExecutionOrder(completedOrder));
521         final Collection<OnPhaseFinished> listeners = phaseListeners.get(phase);
522         if (!listeners.isEmpty()) {
523             runPhaseListeners(phase, listeners);
524         }
525     }
526
527     private void summarizeSubstatementPolicy() {
528         if (definition().support().copyPolicy() == CopyPolicy.EXACT_REPLICA || noSensitiveSubstatements()) {
529             setAllSubstatementsContextIndependent();
530         }
531     }
532
533     /**
534      * Determine whether any substatements are copy-sensitive as determined by {@link StatementSupport#copyPolicy()}.
535      * Only {@link CopyPolicy#CONTEXT_INDEPENDENT}, {@link CopyPolicy#EXACT_REPLICA} and {@link CopyPolicy#IGNORE} are
536      * copy-insensitive. Note that statements which are not {@link StmtContext#isSupportedToBuildEffective()} are all
537      * considered copy-insensitive.
538      *
539      * <p>
540      * Implementations are expected to call {@link #noSensitiveSubstatements()} to actually traverse substatement sets.
541      *
542      * @return True if no substatements require copy-sensitive handling
543      */
544     abstract boolean noSensitiveSubstatements();
545
546     /**
547      * Determine whether any of the provided substatements are context-sensitive for purposes of implementing
548      * {@link #noSensitiveSubstatements()}.
549      *
550      * @param substatements Substatements to check
551      * @return True if no substatements require context-sensitive handling
552      */
553     static boolean noSensitiveSubstatements(final Collection<? extends ReactorStmtCtx<?, ?, ?>> substatements) {
554         for (ReactorStmtCtx<?, ?, ?> stmt : substatements) {
555             if (stmt.isSupportedToBuildEffective()) {
556                 if (!stmt.allSubstatementsContextIndependent()) {
557                     // This is a recursive property
558                     return false;
559                 }
560
561                 switch (stmt.definition().support().copyPolicy()) {
562                     case CONTEXT_INDEPENDENT:
563                     case EXACT_REPLICA:
564                     case IGNORE:
565                         break;
566                     default:
567                         return false;
568                 }
569             }
570         }
571         return true;
572     }
573
574     private void runPhaseListeners(final ModelProcessingPhase phase, final Collection<OnPhaseFinished> listeners) {
575         final Iterator<OnPhaseFinished> listener = listeners.iterator();
576         while (listener.hasNext()) {
577             final OnPhaseFinished next = listener.next();
578             if (next.phaseFinished(this, phase)) {
579                 listener.remove();
580             }
581         }
582
583         if (listeners.isEmpty()) {
584             phaseListeners.removeAll(phase);
585             if (phaseListeners.isEmpty()) {
586                 phaseListeners = ImmutableMultimap.of();
587             }
588         }
589     }
590
591     @Override
592     final StatementDefinitionContext<A, D, E> definition() {
593         return definition;
594     }
595
596     final <K, V, N extends ParserNamespace<K, V>> void onNamespaceItemAddedAction(final Class<N> type, final K key,
597             final OnNamespaceItemAdded listener) {
598         final Object potential = getFromNamespace(type, key);
599         if (potential != null) {
600             LOG.trace("Listener on {} key {} satisfied immediately", type, key);
601             listener.namespaceItemAdded(this, type, key, potential);
602             return;
603         }
604
605         getBehaviour(type).addListener(new KeyedValueAddedListener<>(this, key) {
606             @Override
607             void onValueAdded(final Object value) {
608                 listener.namespaceItemAdded(StatementContextBase.this, type, key, value);
609             }
610         });
611     }
612
613     final <K, V, N extends ParserNamespace<K, V>> void onNamespaceItemAddedAction(final Class<N> type,
614             final ModelProcessingPhase phase, final NamespaceKeyCriterion<K> criterion,
615             final OnNamespaceItemAdded listener) {
616         final Optional<Entry<K, V>> existing = getFromNamespace(type, criterion);
617         if (existing.isPresent()) {
618             final Entry<K, V> entry = existing.get();
619             LOG.debug("Listener on {} criterion {} found a pre-existing match: {}", type, criterion, entry);
620             waitForPhase(entry.getValue(), type, phase, criterion, listener);
621             return;
622         }
623
624         final NamespaceBehaviourWithListeners<K, V, N> behaviour = getBehaviour(type);
625         behaviour.addListener(new PredicateValueAddedListener<K, V>(this) {
626             @Override
627             boolean onValueAdded(final K key, final V value) {
628                 if (criterion.match(key)) {
629                     LOG.debug("Listener on {} criterion {} matched added key {}", type, criterion, key);
630                     waitForPhase(value, type, phase, criterion, listener);
631                     return true;
632                 }
633
634                 return false;
635             }
636         });
637     }
638
639     final <K, V, N extends ParserNamespace<K, V>> void selectMatch(final Class<N> type,
640             final NamespaceKeyCriterion<K> criterion, final OnNamespaceItemAdded listener) {
641         final Optional<Entry<K, V>> optMatch = getFromNamespace(type, criterion);
642         checkState(optMatch.isPresent(), "Failed to find a match for criterion %s in namespace %s node %s", criterion,
643             type, this);
644         final Entry<K, V> match = optMatch.get();
645         listener.namespaceItemAdded(StatementContextBase.this, type, match.getKey(), match.getValue());
646     }
647
648     final <K, V, N extends ParserNamespace<K, V>> void waitForPhase(final Object value, final Class<N> type,
649             final ModelProcessingPhase phase, final NamespaceKeyCriterion<K> criterion,
650             final OnNamespaceItemAdded listener) {
651         ((StatementContextBase<?, ? ,?>) value).addPhaseCompletedListener(phase,
652             (context, phaseCompleted) -> {
653                 selectMatch(type, criterion, listener);
654                 return true;
655             });
656     }
657
658     private <K, V, N extends ParserNamespace<K, V>> NamespaceBehaviourWithListeners<K, V, N> getBehaviour(
659             final Class<N> type) {
660         final NamespaceBehaviour<K, V, N> behaviour = getBehaviourRegistry().getNamespaceBehaviour(type);
661         checkArgument(behaviour instanceof NamespaceBehaviourWithListeners, "Namespace %s does not support listeners",
662             type);
663
664         return (NamespaceBehaviourWithListeners<K, V, N>) behaviour;
665     }
666
667     private static <T> Multimap<ModelProcessingPhase, T> newMultimap() {
668         return Multimaps.newListMultimap(new EnumMap<>(ModelProcessingPhase.class), () -> new ArrayList<>(1));
669     }
670
671     /**
672      * Adds {@link OnPhaseFinished} listener for a {@link ModelProcessingPhase} end. If the base has already completed
673      * the listener is notified immediately.
674      *
675      * @param phase requested completion phase
676      * @param listener listener to invoke
677      * @throws NullPointerException if any of the arguments is null
678      */
679     void addPhaseCompletedListener(final ModelProcessingPhase phase, final OnPhaseFinished listener) {
680         requireNonNull(phase, "Statement context processing phase cannot be null");
681         requireNonNull(listener, "Statement context phase listener cannot be null");
682
683         ModelProcessingPhase finishedPhase = ModelProcessingPhase.ofExecutionOrder(executionOrder);
684         while (finishedPhase != null) {
685             if (phase.equals(finishedPhase)) {
686                 listener.phaseFinished(this, finishedPhase);
687                 return;
688             }
689             finishedPhase = finishedPhase.getPreviousPhase();
690         }
691         if (phaseListeners.isEmpty()) {
692             phaseListeners = newMultimap();
693         }
694
695         phaseListeners.put(phase, listener);
696     }
697
698     /**
699      * Adds a {@link ContextMutation} to a {@link ModelProcessingPhase}.
700      *
701      * @throws IllegalStateException when the mutation was registered after phase was completed
702      */
703     final void addMutation(final ModelProcessingPhase phase, final ContextMutation mutation) {
704         checkState(executionOrder < phase.executionOrder(), "Mutation registered after phase was completed at: %s",
705             sourceReference());
706
707         if (phaseMutation.isEmpty()) {
708             phaseMutation = newMultimap();
709         }
710         phaseMutation.put(phase, mutation);
711     }
712
713     final void removeMutation(final ModelProcessingPhase phase, final ContextMutation mutation) {
714         if (!phaseMutation.isEmpty()) {
715             phaseMutation.remove(phase, mutation);
716             cleanupPhaseMutation();
717         }
718     }
719
720     @Override
721     public final <K, KT extends K, N extends StatementNamespace<K, ?, ?>> void addContext(
722             final Class<@NonNull N> namespace, final KT key,final StmtContext<?, ?, ?> stmt) {
723         addContextToNamespace(namespace, key, stmt);
724     }
725
726     @Override
727     public final Optional<? extends Mutable<?, ?, ?>> copyAsChildOf(final Mutable<?, ?, ?> parent, final CopyType type,
728             final QNameModule targetModule) {
729         checkEffectiveModelCompleted(this);
730         return Optional.ofNullable(copyAsChildOfImpl(parent, type, targetModule));
731     }
732
733     private @Nullable ReactorStmtCtx<A, D, E> copyAsChildOfImpl(final Mutable<?, ?, ?> parent, final CopyType type,
734             final QNameModule targetModule) {
735         final StatementSupport<A, D, E> support = definition.support();
736         final CopyPolicy policy = support.copyPolicy();
737         switch (policy) {
738             case EXACT_REPLICA:
739                 return replicaAsChildOf(parent);
740             case CONTEXT_INDEPENDENT:
741                 if (allSubstatementsContextIndependent()) {
742                     return replicaAsChildOf(parent);
743                 }
744
745                 // fall through
746             case DECLARED_COPY:
747                 // FIXME: ugly cast
748                 return (ReactorStmtCtx<A, D, E>) parent.childCopyOf(this, type, targetModule);
749             case IGNORE:
750                 return null;
751             case REJECT:
752                 throw new IllegalStateException("Statement " + support.getPublicView() + " should never be copied");
753             default:
754                 throw new IllegalStateException("Unhandled policy " + policy);
755         }
756     }
757
758     @Override
759     final ReactorStmtCtx<?, ?, ?> asEffectiveChildOf(final StatementContextBase<?, ?, ?> parent, final CopyType type,
760             final QNameModule targetModule) {
761         final ReactorStmtCtx<A, D, E> copy = copyAsChildOfImpl(parent, type, targetModule);
762         if (copy == null) {
763             // The statement fizzled, this should never happen, perhaps a verify()?
764             return null;
765         }
766
767         parent.ensureCompletedExecution(copy);
768         return canReuseCurrent(copy) ? this : copy;
769     }
770
771     private boolean canReuseCurrent(final @NonNull ReactorStmtCtx<A, D, E> copy) {
772         // Defer to statement factory to see if we can reuse this object. If we can and have only context-independent
773         // substatements we can reuse the object. More complex cases are handled indirectly via the copy.
774         return definition.getFactory().canReuseCurrent(copy, this, buildEffective().effectiveSubstatements())
775             && allSubstatementsContextIndependent();
776     }
777
778     @Override
779     public final Mutable<?, ?, ?> childCopyOf(final StmtContext<?, ?, ?> stmt, final CopyType type,
780             final QNameModule targetModule) {
781         checkEffectiveModelCompleted(stmt);
782         if (stmt instanceof StatementContextBase) {
783             return childCopyOf((StatementContextBase<?, ?, ?>) stmt, type, targetModule);
784         } else if (stmt instanceof ReplicaStatementContext) {
785             return ((ReplicaStatementContext<?, ?, ?>) stmt).replicaAsChildOf(this);
786         } else {
787             throw new IllegalArgumentException("Unsupported statement " + stmt);
788         }
789     }
790
791     private <X, Y extends DeclaredStatement<X>, Z extends EffectiveStatement<X, Y>> Mutable<X, Y, Z> childCopyOf(
792             final StatementContextBase<X, Y, Z> original, final CopyType type, final QNameModule targetModule) {
793         final var implicitParent = definition.getImplicitParentFor(this, original.publicDefinition());
794
795         final StatementContextBase<X, Y, Z> result;
796         final InferredStatementContext<X, Y, Z> copy;
797
798         if (implicitParent.isPresent()) {
799             result = new UndeclaredStmtCtx(this, implicitParent.orElseThrow(), original, type);
800
801             final CopyType childCopyType = switch (type) {
802                 case ADDED_BY_AUGMENTATION -> CopyType.ORIGINAL;
803                 case ADDED_BY_USES_AUGMENTATION -> CopyType.ADDED_BY_USES;
804                 case ADDED_BY_USES, ORIGINAL -> type;
805             };
806             copy = new InferredStatementContext<>(result, original, childCopyType, type, targetModule);
807             result.addEffectiveSubstatement(copy);
808             result.definition.onStatementAdded(result);
809         } else {
810             result = copy = new InferredStatementContext<>(this, original, type, type, targetModule);
811         }
812
813         original.definition.onStatementAdded(copy);
814         return result;
815     }
816
817     @Override
818     final ReplicaStatementContext<A, D, E> replicaAsChildOf(final StatementContextBase<?, ?, ?> parent) {
819         return new ReplicaStatementContext<>(parent, this);
820     }
821
822     private static void checkEffectiveModelCompleted(final StmtContext<?, ?, ?> stmt) {
823         final ModelProcessingPhase phase = stmt.getCompletedPhase();
824         checkState(phase == ModelProcessingPhase.EFFECTIVE_MODEL,
825                 "Attempted to copy statement %s which has completed phase %s", stmt, phase);
826     }
827
828     @Override
829     public final boolean hasImplicitParentSupport() {
830         return definition.getFactory() instanceof ImplicitParentAwareStatementSupport;
831     }
832
833     @Override
834     public final StmtContext<?, ?, ?> wrapWithImplicit(final StmtContext<?, ?, ?> original) {
835         final var optImplicit = definition.getImplicitParentFor(this, original.publicDefinition());
836         if (optImplicit.isEmpty()) {
837             return original;
838         }
839
840         checkArgument(original instanceof StatementContextBase, "Unsupported original %s", original);
841         final var origBase = (StatementContextBase<?, ?, ?>)original;
842
843         @SuppressWarnings({ "rawtypes", "unchecked" })
844         final UndeclaredStmtCtx<?, ?, ?> result = new UndeclaredStmtCtx(origBase, optImplicit.orElseThrow());
845         result.addEffectiveSubstatement(origBase.reparent(result));
846         result.setCompletedPhase(original.getCompletedPhase());
847         return result;
848     }
849
850     abstract StatementContextBase<A, D, E> reparent(StatementContextBase<?, ?, ?> newParent);
851
852     /**
853      * Indicate that the set of substatements is empty. This is a preferred shortcut to substatement stream filtering.
854      *
855      * @return True if {@link #allSubstatements()} and {@link #allSubstatementsStream()} would return an empty stream.
856      */
857     abstract boolean hasEmptySubstatements();
858 }