You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@daffodil.apache.org by GitBox <gi...@apache.org> on 2020/09/02 14:17:59 UTC

[GitHub] [incubator-daffodil] mbeckerle commented on a change in pull request #410: Stream infoset events while parsing and reduce memory usage

mbeckerle commented on a change in pull request #410:
URL: https://github.com/apache/incubator-daffodil/pull/410#discussion_r482061267



##########
File path: daffodil-core/src/test/scala/org/apache/daffodil/dsom/TestSimpleTypeUnions.scala
##########
@@ -51,7 +51,18 @@ class TestSimpleTypeUnions {
     <xs:simpleType name="oneOrTwo">
       <xs:union memberTypes="ex:int1Type ex:int2Type"/>
     </xs:simpleType>
-    <xs:element name="e1" dfdl:lengthKind="delimited" type="ex:oneOrTwo"/>)
+    <xs:element name="e1" dfdl:lengthKind="delimited" type="ex:oneOrTwo">
+      <xs:annotation>
+        <xs:appinfo source="http://www.ogf.org/dfdl/">
+          <!--
+            this assert always passes, but uses e1 in an expression to prevent
+            the InfosetWalker from freeing it, which allows the tests to
+            inspect runtime internals
+          -->
+          <dfdl:assert test="{ fn:true() or /ex:e1 eq 0 }" />

Review comment:
       Why the fn:true() ? To disable some optimization? Remove or comment why. 

##########
File path: daffodil-runtime1/src/main/scala/org/apache/daffodil/dpath/DFDLXFunctions.scala
##########
@@ -52,7 +43,7 @@ case class DFDLXTrace(recipe: CompiledDPath, msg: String)
         dstate.setCurrentValue(v)
         v.toString()
       }
-      case other: InfosetCommon => "\n" + asXMLString(other)
+      case other: DINode => other.namedQName.toString

Review comment:
       Github won't let me comment on a non-changed line. There is a comment above that says "// if there is no current node there must be a current value." I think that should be an Assert.invariant check. 

##########
File path: daffodil-runtime1/src/main/scala/org/apache/daffodil/infoset/InfosetImpl.scala
##########
@@ -92,21 +92,42 @@ sealed trait DINode {
     else asComplex
 
   def isDefaulted: Boolean
+  def isHidden: Boolean
 
   def children: Stream[DINode]
   def totalElementCount: Long
   def namedQName: NamedQName
   def erd: ElementRuntimeData
 
+  def infosetWalkerBlockCount: Int
+
   /**
    * Can treat any DINode, even simple ones, as a container of other nodes.
    * This simplifies walking an infoset.
    */
   def contents: IndexedSeq[DINode]
   final def numChildren = contents.length
 
-  def visit(handler: InfosetOutputter, removeHidden: Boolean = true): Unit
+  /**
+   * Returns the DINode that contains this DINode. Note that this behavior is
+   * slightly different from the "parent". The parent method always returns a
+   * DIComplex, even if this DINode is in an array. This usually makes sense
+   * and is what the caller intended when they call parent.
+   *
+   * However, we sometimes want the exact DINode that contains this DINode. So
+   * if this DINode is a DIElement in an array, then this returns the DIArray.
+   * Otherwise (i.e. DIArray or non-array DIElement) then it should return the
+   * DIComplex parent.

Review comment:
       Add to comment what happens if you call this on the root. (Even if that's just "it's an error to call this..."

##########
File path: daffodil-runtime1/src/main/scala/org/apache/daffodil/infoset/InfosetImpl.scala
##########
@@ -812,6 +833,15 @@ sealed trait DIElement
   override final def namedQName = erd.namedQName
   override final def trd = erd
 
+  /**
+   * Used to prevent the infoset walker from walking into an infoset element.
+   * This is generally increment when marks are added and decremented when they

Review comment:
       incremented

##########
File path: daffodil-runtime1/src/main/scala/org/apache/daffodil/infoset/InfosetWalker.scala
##########
@@ -0,0 +1,451 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.daffodil.infoset
+
+import org.apache.daffodil.exceptions.Assert
+import org.apache.daffodil.util.MStackOfInt
+import org.apache.daffodil.util.Maybe
+import org.apache.daffodil.util.Maybe.One
+import org.apache.daffodil.util.Maybe.Nope
+
+object InfosetWalker {
+
+  /**
+   * Create an infoset walker starting with a specified DINode. If the caller
+   * starts the InfosetWalker at a specified DINode that is not a DIDocument
+   * (which is often the case with the debugger) the InfosetWalker will walk
+   * the node and its children, but will not walk any siblings of the node.
+   * Regardless whether the DINode is a DIDocument or something else, the
+   * InfosetWalker will still always call the start/endDocument functions of
+   * the InfosetOutputter.
+   *
+   * @param root
+   *
+   *   The DINode to start walking the infoset
+   *
+   * @param outputter
+   *
+   *   The InfosetOutputter to output events while walking the infoset
+   *
+   * @param walkHidden
+   *
+   *   Whether or not to walk infoset elements that are considered hidden. This
+   *   should usually only be set to true while debugging
+   *
+   * @param ignoreBlocks
+   *
+   *   Whether or not to ignore blocks when walking the infoset, which are used
+   *   to prevent creation of infoset events that might be backtracked. This
+   *   should usually only be set to true while debugging
+   *
+   * @param removeUnneeded
+   *
+   *   Whether or not to remove infoset nodes once it is determined that they
+   *   will no longer be used by Daffodil. This should usually be set to true
+   *   except while debugging
+   */
+  def apply(
+    root: DINode,
+    outputter: InfosetOutputter,
+    walkHidden: Boolean,
+    ignoreBlocks: Boolean,
+    removeUnneeded: Boolean): InfosetWalker = {
+
+    // Determine the container of the root node and the index in which it
+    // appears in that node
+    val (startingContainerNode, startingContainerIndex) = root match {
+      case d: DIDocument => {
+        // We want to start at the zero'th child index of the document
+        (d, 0)
+      }
+      case _ => {
+        // This case should only be hit when using the debugger to start
+        // walking at a node that isn't the root DIDocument. This gets the
+        // container of the root node to start at and finds the index in that
+        // container
+        (root.containerNode, root.containerNode.contents.indexOf(root))
+      }
+    }
+    new InfosetWalker(
+      startingContainerNode,
+      startingContainerIndex,
+      outputter,
+      walkHidden,
+      ignoreBlocks,
+      removeUnneeded)
+  }
+
+}
+
+/**
+ * The purpose of this class is to walk Daffodil's internal infoset
+ * representation (i.e. DINodes), starting at a specified DINode, and call the
+ * appropriate functions on the InfosetOutputter. The InfosetOutputter provided
+ * to this class determines how the internal infoset is projectted to the user
+ * (e.g. XML, json, SAX events).
+ *
+ * Calling the walk() method causes the InfosetWalker to begin this process. At
+ * any point as determined by the walker, it may pause the walk. Thus, the
+ * entire infoset is not guaranteed to have been walked when the walk() method
+ * returns. In fact, it is possible that no progress could be made. It is up to
+ * the caller to call walk() at appropriate times where it is likely that a
+ * walk could make progress.
+ *
+ * The isFinished() method can be used to determine if the walker walked the
+ * entire infoset or not.
+ *
+ * @param startingContainerNode
+ *
+ *   The container DINode of the element to start at. This should be either a
+ *   DIComplex, DIDocument, or DIArray. DISimple is technically allowed, but
+ *   will not create any useful events as it contains no children.
+ *
+ * @param startingContainerIndex
+ *
+ *   The child index of the element in the startingContainerNode
+ *
+ * @param outputter
+ *
+ *   The InfosetOutputter to output events while walking the infoset
+ *
+ * @param walkHidden
+ *
+ *   Whether or not to walk infoset elements that are considered hidden. This
+ *   should usually only be set to true while debugging
+ *
+ * @param ignoreBlocks
+ *
+ *   Whether or not to ignore blocks when walking the infoset, which are used
+ *   to prevent creation of infoset events that might be backtracked. This
+ *   should usually only be set to true while debugging
+ *
+ * @param removeUnneeded
+ *
+ *   Whether or not to remove infoset nodes once it is determined that they
+ *   will no longer be used by Daffodil. This should usually be set to true
+ *   except while debugging
+ */
+class InfosetWalker(
+  startingContainerNode: DINode,
+  startingContainerIndex: Int,
+  val outputter: InfosetOutputter,
+  walkHidden: Boolean,
+  ignoreBlocks: Boolean,
+  removeUnneeded: Boolean) {
+
+  /**
+   * These two pieces of mutable state are all that is needed to keep track of
+   * where we are in the infoset. The element that the walker will output an
+   * event for when step() is called is referenced by its container DINode
+   * (either a DIComplex/DIDocument or DIArray) which is stored in
+   * containerNode, and its index within that containing node (which is the
+   * value on the top of the containerIndexStack). Once step() creates the
+   * appropriate event for the element, it will mutate this state so the next
+   * call to step creates events for the next element in the infoset.
+   *
+   * To step to the next sibling of an element, we only need to increment the
+   * index on the top of the stack, since siblings have the same container.
+   *
+   * To step into a DIComplex/DIArray, we mutate containerNode to be the
+   * DIComplex/DIArray we want to step into and push a zero onto the container
+   * index stack so that the next element is the zero'th child of that
+   * container.
+   *
+   * To step out of a DIComplex/DIArray, we mutate containerNode to be the
+   * container of the containerNode and pop off the top of the stack. We then
+   * can perform the logic to step to the next sibling.
+   *
+   * Special helper functions are created to make the above logic more clear.
+   *
+   * Note that we initialize the top of the container index stack with one less
+   * than the starting container index. This lets the getNextStep function know
+   * that we have not yet started the document. Once the document is started,
+   * we increment this value on the top of the stack so that the starting index
+   * is correct.
+   */
+  private var containerNode: DINode = startingContainerNode
+  private var containerIndexStack: MStackOfInt = {
+    val stack = MStackOfInt()
+    stack.push(startingContainerIndex - 1)
+    stack
+  }
+
+  private var finished = false
+
+  /**
+   * Determine if the walker has finished walking.
+   */
+  def isFinished = finished
+
+  /**
+   * Take zero or more steps in the infoset. This may or may not walk the
+   * entire infoset. If isFinished returns false, walk can be called again to
+   * continue attempting to walk the infoset where it left off. Because this is
+   * not guaranteed to make any progress, the caller should attempt to only
+   * call walk() when infoset state has changed such that progress may be
+   * possible.
+   *
+   * It is an error to call walk() if isFinished returns true
+   */
+  def walk(): Unit = {
+    Assert.usage(!finished)
+
+    var maybeNextStep: Maybe[InfosetWalkerStep] = Nope
+    while ({
+      maybeNextStep = getNextStep()
+      maybeNextStep.isDefined
+    }) {
+      maybeNextStep.get.step()
+    }
+  }
+
+  /**
+   * Determine the next step to take, if any
+   */
+  @inline

Review comment:
       REmove inline. This is too big and complex to want to actually inline it isn't it?

##########
File path: daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/parsers/SequenceParserBases.scala
##########
@@ -82,6 +83,18 @@ abstract class SequenceParserBase(
 
     val infosetIndexStart = pstate.infoset.asInstanceOf[DIComplex].childNodes.size
 
+    if (!isOrdered) {
+      // If this is an unordered sequence, upon completion of parsing all the
+      // elements we will reorder the elements into schema definition order.
+      // This means that we cannot let the infoset walker walk any of the
+      // elements while we are parsing because their event order might change.
+      // To ensure we don't walk, add a block on the parent of the infoset
+      // elements. The infoset walker will inspect this to see if it should
+      // walk any children. We'll remove this block once the unordered sequence
+      // is complete.
+      pstate.infoset.infosetWalkerBlockCount += 1

Review comment:
       So this means we can't incrementally parse an unordered sequence at all, even if it arrives in perfect order. I'm fine with that. We need to accumulate these limitations in a user's guide/note so as to explain why schemas can stream or not. 
   
   We might even want a tunable that turns on "streamability" warnings, and issue SDW for non-streamable constructs in schemas. 

##########
File path: daffodil-runtime1/src/main/scala/org/apache/daffodil/infoset/InfosetImpl.scala
##########
@@ -812,6 +833,15 @@ sealed trait DIElement
   override final def namedQName = erd.namedQName
   override final def trd = erd
 
+  /**
+   * Used to prevent the infoset walker from walking into an infoset element.
+   * This is generally increment when marks are added and decremented when they
+   * are discarded. The idea being that we do not want to walk into infoset
+   * elements if there could be backtracking. Though this is not strictly the
+   * only time blocks may be added.

Review comment:
       What are other times?

##########
File path: daffodil-core/src/test/scala/org/apache/daffodil/dsom/TestSimpleTypeUnions.scala
##########
@@ -270,11 +292,22 @@ class TestSimpleTypeUnions {
         <xs:pattern value="foo[1234]bar"/>
       </xs:restriction>
     </xs:simpleType>
-    <xs:element name="e1" dfdl:lengthKind="delimited" type="ex:foo3bar"/>)
+    <xs:element name="e1" dfdl:lengthKind="delimited" type="ex:foo3bar">
+      <xs:annotation>
+        <xs:appinfo source="http://www.ogf.org/dfdl/">
+          <!--
+            this assert always passes, but uses e1 in an expression to prevent
+            the InfosetWalker from freeing it, which allows the tests to
+            inspect runtime internals
+          -->
+          <dfdl:assert test="{ fn:true() or /ex:e1 eq '' }" />

Review comment:
       same as prior comment.

##########
File path: daffodil-runtime1/src/main/scala/org/apache/daffodil/infoset/InfosetImpl.scala
##########
@@ -92,21 +92,42 @@ sealed trait DINode {
     else asComplex
 
   def isDefaulted: Boolean
+  def isHidden: Boolean
 
   def children: Stream[DINode]
   def totalElementCount: Long
   def namedQName: NamedQName
   def erd: ElementRuntimeData
 
+  def infosetWalkerBlockCount: Int
+
   /**
    * Can treat any DINode, even simple ones, as a container of other nodes.
    * This simplifies walking an infoset.
    */
   def contents: IndexedSeq[DINode]
   final def numChildren = contents.length
 
-  def visit(handler: InfosetOutputter, removeHidden: Boolean = true): Unit
+  /**
+   * Returns the DINode that contains this DINode. Note that this behavior is
+   * slightly different from the "parent". The parent method always returns a
+   * DIComplex, even if this DINode is in an array. This usually makes sense
+   * and is what the caller intended when they call parent.
+   *
+   * However, we sometimes want the exact DINode that contains this DINode. So
+   * if this DINode is a DIElement in an array, then this returns the DIArray.
+   * Otherwise (i.e. DIArray or non-array DIElement) then it should return the
+   * DIComplex parent.
+   */
+  def containerNode: DINode
 
+  /**
+   * Free memory associated with the child at a specified index. Note that this
+   * should only free children if they are not actually needed anymore. It can
+   * assume that the infoset events have been created, but if, for example, a
+   * child is used in DPath expressions this should not free the child.

Review comment:
       Rename to freeChildIfNoLongerNeeded ? 

##########
File path: daffodil-lib/src/main/scala/org/apache/daffodil/util/MStack.scala
##########
@@ -269,9 +269,9 @@ protected abstract class MStack[@specialized T] private[util] (
    *
    *  @return the element on top of the stack.
    */
-  @inline final def top: T = table(index - 1).asInstanceOf[T]
+  @inline final def top: T = table(index - 1)

Review comment:
       It would be nice if Scala told us about unnecessary casts like this. But it seems it does not. 

##########
File path: daffodil-runtime1/src/main/scala/org/apache/daffodil/debugger/InteractiveDebugger.scala
##########
@@ -1512,12 +1517,23 @@ class InteractiveDebugger(runner: InteractiveDebuggerRunner, eCompilers: Express
               }
             }
           }
+          // if infoset is a DIDocument but that DIDocument has no children
+          // then consider this as not having an infoset. set infoset to null
+          infoset = infoset match {
+            case d: DIDocument if d.contents.size == 0 => null

Review comment:
       null ? Should we be using Maybe[InfosetElement] here?

##########
File path: daffodil-runtime1/src/main/scala/org/apache/daffodil/infoset/InfosetImpl.scala
##########
@@ -971,6 +1006,34 @@ final class DIArray(
   override def isSimple = false
   override def isComplex = false
 
+  // Parsers don't actually call setHidden on DIArrays, only on DIElements. But
+  // DIArrays are always created when there is at least one child element, and
+  // since all children of an array must have the same visibility, we can just
+  // inspect the isHidden property of the first child to determine if the
+  // entire array is hidden.
+  override def isHidden = _contents(0).isHidden
+
+  // Marks/blocks are never taken on arrays, so the infosetWalkerBlockCount
+  // would normally be zero for arrays. The infosetWalkerBlockCount is used to determine
+  // if this array is known to exist or if it could be backtracked. We know
+  // this DIArray exists if the first child is known to exist (i.e. if it has
+  // no blocks).
+  //
+  // Additionally, the infoset walker may delete elements from an array. In

Review comment:
       For maintainability/debuggability, i really think we should not use null here. 
   
   I understand that using a Maybe[DINode] here causes a lot of change. An alternative maybe is we should have a distinguished element/node object that means removed. That may cause a lot of otherwise final methods to be override-able so that one can't accidently start using the removed element. Not sure best approach, but we should consider a way to make it better than just using null.

##########
File path: daffodil-runtime1/src/main/scala/org/apache/daffodil/infoset/InfosetImpl.scala
##########
@@ -971,6 +1006,34 @@ final class DIArray(
   override def isSimple = false
   override def isComplex = false
 
+  // Parsers don't actually call setHidden on DIArrays, only on DIElements. But
+  // DIArrays are always created when there is at least one child element, and
+  // since all children of an array must have the same visibility, we can just
+  // inspect the isHidden property of the first child to determine if the
+  // entire array is hidden.
+  override def isHidden = _contents(0).isHidden
+
+  // Marks/blocks are never taken on arrays, so the infosetWalkerBlockCount
+  // would normally be zero for arrays. The infosetWalkerBlockCount is used to determine
+  // if this array is known to exist or if it could be backtracked. We know
+  // this DIArray exists if the first child is known to exist (i.e. if it has
+  // no blocks).
+  //
+  // Additionally, the infoset walker may delete elements from an array. In
+  // which case _contents(0) may be null. But if _contents(0) is null, that
+  // means there was an element that was known to exist, and it was freed after
+  // its event were created. So that doesn't cause a block.
+  //
+  // We also add the parent block count. This is because when we are
+  // speculatively parsing elements of an array, marks are taken before array
+  // elements are added, which increments the infosetWalkerBlockCount of the
+  // parent of this array. So if the block count of the parent is > 0, then
+  // that means we are speculatively parsing a new child, and so should not
+  // walk elements of the array.

Review comment:
       How does this not stop removal of any array children? We want to be able to walk the earlier parts of an array and prune the children we walk, while the parser speculates on later parts of the array. 

##########
File path: daffodil-runtime1/src/main/scala/org/apache/daffodil/infoset/InfosetWalker.scala
##########
@@ -0,0 +1,451 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.daffodil.infoset
+
+import org.apache.daffodil.exceptions.Assert
+import org.apache.daffodil.util.MStackOfInt
+import org.apache.daffodil.util.Maybe
+import org.apache.daffodil.util.Maybe.One
+import org.apache.daffodil.util.Maybe.Nope
+
+object InfosetWalker {
+
+  /**
+   * Create an infoset walker starting with a specified DINode. If the caller
+   * starts the InfosetWalker at a specified DINode that is not a DIDocument
+   * (which is often the case with the debugger) the InfosetWalker will walk
+   * the node and its children, but will not walk any siblings of the node.
+   * Regardless whether the DINode is a DIDocument or something else, the
+   * InfosetWalker will still always call the start/endDocument functions of
+   * the InfosetOutputter.
+   *
+   * @param root
+   *
+   *   The DINode to start walking the infoset
+   *
+   * @param outputter
+   *
+   *   The InfosetOutputter to output events while walking the infoset
+   *
+   * @param walkHidden
+   *
+   *   Whether or not to walk infoset elements that are considered hidden. This
+   *   should usually only be set to true while debugging
+   *
+   * @param ignoreBlocks
+   *
+   *   Whether or not to ignore blocks when walking the infoset, which are used
+   *   to prevent creation of infoset events that might be backtracked. This
+   *   should usually only be set to true while debugging
+   *
+   * @param removeUnneeded
+   *
+   *   Whether or not to remove infoset nodes once it is determined that they
+   *   will no longer be used by Daffodil. This should usually be set to true
+   *   except while debugging
+   */
+  def apply(
+    root: DINode,
+    outputter: InfosetOutputter,
+    walkHidden: Boolean,
+    ignoreBlocks: Boolean,
+    removeUnneeded: Boolean): InfosetWalker = {
+
+    // Determine the container of the root node and the index in which it
+    // appears in that node
+    val (startingContainerNode, startingContainerIndex) = root match {
+      case d: DIDocument => {
+        // We want to start at the zero'th child index of the document
+        (d, 0)
+      }
+      case _ => {
+        // This case should only be hit when using the debugger to start
+        // walking at a node that isn't the root DIDocument. This gets the
+        // container of the root node to start at and finds the index in that
+        // container
+        (root.containerNode, root.containerNode.contents.indexOf(root))
+      }
+    }
+    new InfosetWalker(
+      startingContainerNode,
+      startingContainerIndex,
+      outputter,
+      walkHidden,
+      ignoreBlocks,
+      removeUnneeded)
+  }
+
+}
+
+/**
+ * The purpose of this class is to walk Daffodil's internal infoset
+ * representation (i.e. DINodes), starting at a specified DINode, and call the
+ * appropriate functions on the InfosetOutputter. The InfosetOutputter provided
+ * to this class determines how the internal infoset is projectted to the user
+ * (e.g. XML, json, SAX events).
+ *
+ * Calling the walk() method causes the InfosetWalker to begin this process. At
+ * any point as determined by the walker, it may pause the walk. Thus, the
+ * entire infoset is not guaranteed to have been walked when the walk() method
+ * returns. In fact, it is possible that no progress could be made. It is up to
+ * the caller to call walk() at appropriate times where it is likely that a
+ * walk could make progress.
+ *
+ * The isFinished() method can be used to determine if the walker walked the
+ * entire infoset or not.
+ *
+ * @param startingContainerNode
+ *
+ *   The container DINode of the element to start at. This should be either a
+ *   DIComplex, DIDocument, or DIArray. DISimple is technically allowed, but
+ *   will not create any useful events as it contains no children.
+ *
+ * @param startingContainerIndex
+ *
+ *   The child index of the element in the startingContainerNode
+ *
+ * @param outputter
+ *
+ *   The InfosetOutputter to output events while walking the infoset
+ *
+ * @param walkHidden
+ *
+ *   Whether or not to walk infoset elements that are considered hidden. This
+ *   should usually only be set to true while debugging
+ *
+ * @param ignoreBlocks
+ *
+ *   Whether or not to ignore blocks when walking the infoset, which are used
+ *   to prevent creation of infoset events that might be backtracked. This
+ *   should usually only be set to true while debugging
+ *
+ * @param removeUnneeded
+ *
+ *   Whether or not to remove infoset nodes once it is determined that they
+ *   will no longer be used by Daffodil. This should usually be set to true
+ *   except while debugging
+ */
+class InfosetWalker(

Review comment:
       private constructor since you have factory object?

##########
File path: daffodil-runtime1/src/main/scala/org/apache/daffodil/infoset/InfosetWalker.scala
##########
@@ -0,0 +1,451 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.daffodil.infoset
+
+import org.apache.daffodil.exceptions.Assert
+import org.apache.daffodil.util.MStackOfInt
+import org.apache.daffodil.util.Maybe
+import org.apache.daffodil.util.Maybe.One
+import org.apache.daffodil.util.Maybe.Nope
+
+object InfosetWalker {
+
+  /**
+   * Create an infoset walker starting with a specified DINode. If the caller
+   * starts the InfosetWalker at a specified DINode that is not a DIDocument
+   * (which is often the case with the debugger) the InfosetWalker will walk
+   * the node and its children, but will not walk any siblings of the node.
+   * Regardless whether the DINode is a DIDocument or something else, the
+   * InfosetWalker will still always call the start/endDocument functions of
+   * the InfosetOutputter.
+   *
+   * @param root
+   *
+   *   The DINode to start walking the infoset
+   *
+   * @param outputter
+   *
+   *   The InfosetOutputter to output events while walking the infoset
+   *
+   * @param walkHidden
+   *
+   *   Whether or not to walk infoset elements that are considered hidden. This
+   *   should usually only be set to true while debugging
+   *
+   * @param ignoreBlocks
+   *
+   *   Whether or not to ignore blocks when walking the infoset, which are used
+   *   to prevent creation of infoset events that might be backtracked. This
+   *   should usually only be set to true while debugging
+   *
+   * @param removeUnneeded
+   *
+   *   Whether or not to remove infoset nodes once it is determined that they
+   *   will no longer be used by Daffodil. This should usually be set to true
+   *   except while debugging
+   */
+  def apply(
+    root: DINode,
+    outputter: InfosetOutputter,
+    walkHidden: Boolean,
+    ignoreBlocks: Boolean,
+    removeUnneeded: Boolean): InfosetWalker = {
+
+    // Determine the container of the root node and the index in which it
+    // appears in that node
+    val (startingContainerNode, startingContainerIndex) = root match {
+      case d: DIDocument => {
+        // We want to start at the zero'th child index of the document
+        (d, 0)
+      }
+      case _ => {
+        // This case should only be hit when using the debugger to start
+        // walking at a node that isn't the root DIDocument. This gets the
+        // container of the root node to start at and finds the index in that
+        // container
+        (root.containerNode, root.containerNode.contents.indexOf(root))
+      }
+    }
+    new InfosetWalker(
+      startingContainerNode,
+      startingContainerIndex,
+      outputter,
+      walkHidden,
+      ignoreBlocks,
+      removeUnneeded)
+  }
+
+}
+
+/**
+ * The purpose of this class is to walk Daffodil's internal infoset
+ * representation (i.e. DINodes), starting at a specified DINode, and call the
+ * appropriate functions on the InfosetOutputter. The InfosetOutputter provided
+ * to this class determines how the internal infoset is projectted to the user
+ * (e.g. XML, json, SAX events).
+ *
+ * Calling the walk() method causes the InfosetWalker to begin this process. At
+ * any point as determined by the walker, it may pause the walk. Thus, the
+ * entire infoset is not guaranteed to have been walked when the walk() method
+ * returns. In fact, it is possible that no progress could be made. It is up to
+ * the caller to call walk() at appropriate times where it is likely that a
+ * walk could make progress.
+ *
+ * The isFinished() method can be used to determine if the walker walked the
+ * entire infoset or not.
+ *
+ * @param startingContainerNode
+ *
+ *   The container DINode of the element to start at. This should be either a
+ *   DIComplex, DIDocument, or DIArray. DISimple is technically allowed, but
+ *   will not create any useful events as it contains no children.
+ *
+ * @param startingContainerIndex
+ *
+ *   The child index of the element in the startingContainerNode
+ *
+ * @param outputter
+ *
+ *   The InfosetOutputter to output events while walking the infoset
+ *
+ * @param walkHidden
+ *
+ *   Whether or not to walk infoset elements that are considered hidden. This
+ *   should usually only be set to true while debugging
+ *
+ * @param ignoreBlocks
+ *
+ *   Whether or not to ignore blocks when walking the infoset, which are used
+ *   to prevent creation of infoset events that might be backtracked. This
+ *   should usually only be set to true while debugging
+ *
+ * @param removeUnneeded
+ *
+ *   Whether or not to remove infoset nodes once it is determined that they
+ *   will no longer be used by Daffodil. This should usually be set to true
+ *   except while debugging
+ */
+class InfosetWalker(
+  startingContainerNode: DINode,
+  startingContainerIndex: Int,
+  val outputter: InfosetOutputter,
+  walkHidden: Boolean,
+  ignoreBlocks: Boolean,
+  removeUnneeded: Boolean) {
+
+  /**
+   * These two pieces of mutable state are all that is needed to keep track of
+   * where we are in the infoset. The element that the walker will output an
+   * event for when step() is called is referenced by its container DINode
+   * (either a DIComplex/DIDocument or DIArray) which is stored in
+   * containerNode, and its index within that containing node (which is the
+   * value on the top of the containerIndexStack). Once step() creates the
+   * appropriate event for the element, it will mutate this state so the next
+   * call to step creates events for the next element in the infoset.
+   *
+   * To step to the next sibling of an element, we only need to increment the
+   * index on the top of the stack, since siblings have the same container.
+   *
+   * To step into a DIComplex/DIArray, we mutate containerNode to be the
+   * DIComplex/DIArray we want to step into and push a zero onto the container
+   * index stack so that the next element is the zero'th child of that
+   * container.
+   *
+   * To step out of a DIComplex/DIArray, we mutate containerNode to be the
+   * container of the containerNode and pop off the top of the stack. We then
+   * can perform the logic to step to the next sibling.
+   *
+   * Special helper functions are created to make the above logic more clear.
+   *
+   * Note that we initialize the top of the container index stack with one less
+   * than the starting container index. This lets the getNextStep function know
+   * that we have not yet started the document. Once the document is started,
+   * we increment this value on the top of the stack so that the starting index
+   * is correct.
+   */
+  private var containerNode: DINode = startingContainerNode
+  private var containerIndexStack: MStackOfInt = {
+    val stack = MStackOfInt()
+    stack.push(startingContainerIndex - 1)
+    stack
+  }
+
+  private var finished = false
+
+  /**
+   * Determine if the walker has finished walking.
+   */
+  def isFinished = finished
+
+  /**
+   * Take zero or more steps in the infoset. This may or may not walk the
+   * entire infoset. If isFinished returns false, walk can be called again to
+   * continue attempting to walk the infoset where it left off. Because this is
+   * not guaranteed to make any progress, the caller should attempt to only
+   * call walk() when infoset state has changed such that progress may be
+   * possible.
+   *
+   * It is an error to call walk() if isFinished returns true
+   */
+  def walk(): Unit = {
+    Assert.usage(!finished)
+
+    var maybeNextStep: Maybe[InfosetWalkerStep] = Nope
+    while ({
+      maybeNextStep = getNextStep()
+      maybeNextStep.isDefined
+    }) {
+      maybeNextStep.get.step()
+    }
+  }
+
+  /**
+   * Determine the next step to take, if any
+   */
+  @inline
+  private def getNextStep(): Maybe[InfosetWalkerStep] = {
+    if (finished) {
+      Nope
+    } else if ((containerNode ne startingContainerNode) || containerIndexStack.top == startingContainerIndex) {
+      // The containerNode is either some child of the starting node (container
+      // node != starting container code) or we are exactly on the starting
+      // node (container node == starting container node && top of index stack
+      // == starting index). So we can potentially take a normal step.
+      //
+      // This doesn't necessarily mean we have a step though, since there may
+      // be PoU's that could backtrack and so we can't create events yet, or
+      // other kinds of blocks could exist. So we need to inspect the infoset
+      // state to determine if we can actually create events for the current
+      // infoset node and take a step.
+      if (ignoreBlocks || canTakeStep()) {
+        One(InfosetWalkerStepMove)
+      } else {
+        Nope
+      }
+    } else {
+      // We only get here if the container node is the starting container node,
+      // but the top of the index is not the index of the root node. This means
+      // the next step is either a start or end step.
+      //
+      // If the top of the index stack is less than the starting index, then we
+      // have not started the document yet (because we initialize the index
+      // stack with one less than the starting index). The next step must be a
+      // start step. The InfosetWalkerStepStart is responsible for creating the
+      // right event and updating the containerIndexStack to reference the
+      // correct starting index.
+      //
+      // Otherwise the top of the index stack must be greater than the starting
+      // index because we have moved passed the starting element index, and
+      // thus the next step is an end step. The InfosetWalkerStepEnd is
+      // responsible for creating the endDocument event and cleaning up state.
+      if (containerIndexStack.top < startingContainerIndex) {
+        One(InfosetWalkerStepStart)
+      } else {
+        One(InfosetWalkerStepEnd)
+      }
+    }
+  }
+
+  private def canTakeStep(): Boolean = {
+
+    if (containerNode.infosetWalkerBlockCount > 0) {
+      // This happens in two cases:
+      //
+      // 1) If we have already walked into a complex type that includes an
+      //    unordered sequence. When we start adding the unordered sequence we
+      //    increment the infosetWalkerBlockCount of the container because we
+      //    can't increment the infosetWalkerBlockCount of the yet to be
+      //    created unordered sequence elements. So we take no further steps
+      //    until the unordered sequence is finished and the block is removed
+      // 2) When we are speculatively parsing elements. When an element may or
+      //    may not exist, we set a point of uncertainty before it is created.
+      //    Setting this mark increments the block count of the container
+      //    element because this optional element does not exist yet.
+      //
+      // In both cases we cannot take a step until the block is removed
+      false
+
+    } else {
+      // no blocks on the container, figure out if we can take a step for the
+      // element and the child index of this container 
+
+      val children = containerNode.contents
+      val childIndex = containerIndexStack.top
+
+      if (childIndex < children.size) {
+        // There is a child element at this index. Taking a step would create
+        // the events for, and moved passed, this element. We cannot do that if
+        // there are any walker blocks associated with this element or if this
+        // is a simple element that is not final.
+        val elem = children(childIndex)
+        if (elem.infosetWalkerBlockCount > 0) {
+          false
+        } else if (elem.isInstanceOf[DISimple] && !elem.isFinal) {
+          false
+        } else {
+          true
+        }
+
+      } else {
+        // There is not currently a child at this index. That either means one
+        // of two thing:
+        //
+        // 1) This container is not final and we are waiting for more children
+        //    to be added to this index. In this case, we cannot make a step
+        //    until children are added or the container is marked as final
+        // 2) This container is final, no more children will be added. So we
+        //    can make a step that will end this container
+        if (containerNode.isFinal) {

Review comment:
       Code style point. I often also find myself writing if statements that just have true and false in the branches because it feels clearer to me. I think in this case your block comment could be distributed into the branches and would motivate the reason not to simplify the if-then-else to just containerNode.isFinal. 

##########
File path: daffodil-runtime1/src/main/scala/org/apache/daffodil/infoset/InfosetImpl.scala
##########
@@ -1325,31 +1391,15 @@ sealed trait DIFinalizable {
 }
 
 /**
- * Complex elements have an array of slots one per named child element.
- *
- * TODO: consider xs:choice - alternatives could share slots, but that would
- * add a lot of complexity, and the nil technique of storing null in a
- * slot to indicate a nilled element only works if we have a positive association
- * of slots to element-bases. If we were to share slots we'd need a different way
- * to indicate nil. A better approach for xs:choice would be a sparse table of
- * slots (whenever there are more than N - some threshold), so that we're not
- * allocating arrays of 200 slots just because there are 200 branches of a choice.
- *
- * A slot stores a Maybe[InfosetCommonMixin]. None means not present (yet, because it
- * hasn't been parsed yet, or it is an optional element (minOccurs 0, maxOccurs 1)
- * and is not present.) One[DISimple] or One[DIComplex] mean a required element
- * is present, or an optional element (minOccurs 0, maxOccurs 1) is present.
- *
- * A slot of a DIComplex should never be null.
- *
- * One[DIArray] means the slot is for a recurring element which can have 2+ instances.
- * The DIArray object's length gives the number of occurrences.
+ * Complex elements have an array of DINodes. Nodes may either be DISimple for
+ * simple types, DIComplex for complex types, or DIArray for arrays or optional
+ * elements. There are never empty or null slots. Nilled elements are

Review comment:
       This comment says "there are never empty or null slots", but we null them when we prune  them don't we?

##########
File path: daffodil-runtime1/src/main/scala/org/apache/daffodil/infoset/InfosetImpl.scala
##########
@@ -1296,16 +1355,23 @@ sealed class DISimple(override val erd: ElementRuntimeData)
 
   override def totalElementCount = 1L
 
-  final def visit(handler: InfosetOutputter, removeHidden: Boolean = true): Unit = {
-    if (!this.isHidden || !removeHidden) {
-      handler.startSimple(this)
-      handler.endSimple(this)
-    }
+  /**
+   * requireFinal is only ever used on unparse, and we never need to require a
+   * simple type to be final during unparse. However, we do need to have an
+   * implementation of this function, as DISimple needs DIFinalizable and
+   * isFinal for parsing and allowing the cleanup of unneeded DINodes.
+   */

Review comment:
       Hmmm. I was thinking we would also be using "final" on simple types to deal with the separation of conversion of the value to the text representation. Did that turn out to be not needed?

##########
File path: daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/parsers/SequenceParserBases.scala
##########
@@ -183,6 +202,35 @@ abstract class SequenceParserBase(
               //
               pstate.mpstate.moveOverOneGroupIndexOnly()
             }
+
+            val newLastChildNode = pstate.infoset.contents.lastOption
+            if (newLastChildNode != savedLastChildNode) {
+              // We have added at least one child to to this complex during
+              // this array loop.
+              //
+              // If the new child is a DIArray, we know this DIArray has at
+              // least one element, but we don't know if we actually added a
+              // new one in this loop or not. So just get the last array
+              // element and set it as final anyways. Note that we need a null
+              // check because if we didn't add an array element this go
+              // around, the last element may have already been walked and
+              // freed and so could now be null.
+              //
+              // If it's not a DIArray, that means it's just an optional
+              // simple/complex and that will get set final below where all
+              // other non-array elements get set as final.
+              newLastChildNode.get match {
+                case a: DIArray => {
+                  val lastArrayElem = a.contents.last

Review comment:
       Not about this line of code, but I realized here that we're pruning elements from arrays, but haven't changed the array representation to not be O(n) in the size of the array. Hence, for a very big array we'll still run into the limitation of max JVM array size, and it will always need to be in memory. If I am correct here then we should park the further enhancement to change the array representation. Whether that will prove important or not, I'm not sure. I imagine that pruning the array elements is the biggest win. 

##########
File path: daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/parsers/Parser.scala
##########
@@ -190,6 +190,8 @@ class ChoiceParser(
     
     var successfullyParsedChildBranch = false
 
+    val savedLastChildNode = pstate.infoset.contents.lastOption

Review comment:
       Probably good to declare types on public val/var. In this case we're using lastOption, so an Option type, which is not usual/preferred in the runtime. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org