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/07/10 14:43:29 UTC

[GitHub] [incubator-daffodil] stevedlawrence commented on a change in pull request #394: Feature Addition: API for traversing DSOM and generating NIFI RecordSchemas from DSOM

stevedlawrence commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r452856611



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.dsom.walker
+
+import java.io.File
+
+import org.apache.daffodil.compiler.Compiler
+import org.apache.daffodil.dsom.{Choice, ComplexTypeBase, ElementBase, GroupRef, ModelGroup, Root, Sequence, SimpleTypeBase, Term}
+
+/**
+ * A class designed to walk the internal representation of a DFDL Schema File.
+ *
+ * There are numerous internal event handlers to be defined by the implementing class; each
+ * corresponds to "encountering" a particular element type in the DFDL Schema file.
+ *
+ * Though recursion is used here to define the walk, it is not advised to use recursion between
+ * these event handlers.  Instead, consider a stack-like structure, as the DFDL Schema structure
+ * as well as the recursive method call structure can be represented by trees.
+ * @param schemaFile an input DFDL Schema file to parse
+ * @tparam T the return type of walkDSOMSchema and onWalkEnd.  This should be a class used to
+ *           represent the structure of the Schema.  You could always make it Unit if you don't
+ *           want these methods to return anything.
+ */
+abstract class AbstractDSOMWalker[T](schemaFile: File) {
+  /**
+   * The root element of the DFDL Schema.  This will be the starting point for the traversal
+   */
+  private final val schemaSetRoot: Root = Compiler().compileFile(schemaFile).sset.root
+
+  /**
+   * Auxiliary constructor to initialize this class with a path to a schema File rather than
+   * a File object directly
+   * @param pathToSchemaFile path to the DFDL Schema file desired to be parsed
+   */
+  def this(pathToSchemaFile: String) = this(new File(pathToSchemaFile))
+
+  /**
+   * Method to be called on the beginning of the traversal.  It is recommended to add some
+   * sort of wrapper element to a stack if you're doing a typical stack-based traversal.
+   * @param root the root element of the DFDL Schema
+   */
+  protected def onWalkBegin(root: Root): Unit

Review comment:
       I'm a little hesitant to give a walker implementation direct access to schema compontents. It makes it much more difficult to make changes to these SchemaComponents since we need to be concerned about backwards compatibilty. We sort of have that problem with our InfosetOutputters, I'd like to not repeat that mistake.
   
   I'm wondering if it would be possible to come up with a reasonable list of properties that DSOM walkers actually need and only make those avaialble? I suspect this would be a pretty small list llike namespace, name, type, min/maxOccurs, and maybe a couple more properties?

##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/RecordWalker.scala
##########
@@ -0,0 +1,309 @@
+/*
+ * 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.dsom.walker
+
+import java.io.File
+import java.util
+
+import org.apache.daffodil.dsom.{Choice, ComplexTypeBase, ElementBase, GroupRef, Root, Sequence, SimpleTypeBase, Term}
+import org.apache.nifi.serialization.SimpleRecordSchema
+import org.apache.nifi.serialization.record.{DataType, RecordField, RecordFieldType, RecordSchema, SchemaIdentifier}
+import org.apache.nifi.serialization.record.`type`.RecordDataType
+import scala.collection.mutable.ListBuffer
+
+/**
+ * Direct subclass of the NIFI RecordField.  NIFI doesn't have "Optional" fields, so
+ * eventually this will either be removed from the Schema if an Infoset doesn't have the field,
+ * or it will become a regular RecordField if it does.
+ * @param fieldName the name for the RecordField
+ * @param dataType the NIFI DataType of this RecordField
+ */
+class OptionalRecordField(fieldName: String, dataType: DataType)
+  extends RecordField(fieldName, dataType) {
+  // this is the easiest way to construct it; essentially a "copy constructor" except to denote
+  // that we want to treat it as this special optional subclass.
+  def this(recordField: RecordField) = this(recordField.getFieldName, recordField.getDataType)
+  override def equals(obj: Any): Boolean =
+    obj match {
+      case _: OptionalRecordField => super.equals(obj)
+      case _ => false
+    }
+  override def hashCode(): Int = 31 * super.hashCode() + 1
+  override def toString: String = "Optional" + super.toString
+}
+
+/**
+ * Concrete implementation of the AbstractDSOMWalker abstract class.
+ * This class produces a NIFI RecordSchema that is intended to match the original DFDL file.
+ *
+ * The RecordSchema is built in 3 primary stages:
+ * 1) A tree of SchemaNodes is created as the DFDL file is walked; this walk is performed
+ * through the various event handlers defined in the parent abstract class.
+ * 2) The tree of SchemaNodes undergoes some post-processing, mainly to remove redundant Record wrappers.
+ * 3) The tree of SchemaNodes is converted into a RecordSchema; it is walked recursively within this class.
+ * @param schemaFile an input DFDL Schema file to parse
+ */
+class RecordWalker(schemaFile: File) extends AbstractDSOMWalker[RecordSchema](schemaFile) {

Review comment:
       This is pretty great! I'm surprised at how little code is needed to hook this all together.
   
   However, one thing we need to discuss is where this code actually lives. I think it's safe to safe it shouldn't live in the daffodil-core module. If it does, that means anyone depending on daffodil-core will now pull in a NiFi dependency, which shouldn't be necessary. The DSOM related stuff makes sense to be in core, because it needs to know how to walk the DSOM. But the NiFi specifc code needs to be in a separate module for dependency purposes.
   
   The question, is where does that separate module live? One option would be to create an "integrations" directory, and that could contain separate directories for different kinds of integrations. The jars created from those modules can have separate dependencies so you only need to pull in those dependencies if you're using the integration jars. Something like this probably makes the most sense.
   
   Another thing that plays a role where this lives is this RecordWalker is just one part of NiFi integration. There also needs to be a NiFi Controller/Processor that use this RecordWalker. Ideally, that too would live in the same module so that the result of this module could be a NiFi nar. One issue with that is generally maven is used to build special NiFi nars. Integrating that into the sbt build process might pose some challenges, especially since we have a couple sbt plugins. But maybe we don't want that integration so that building Daffodil doesn't require pulling in dependencies from a bunch of different frameworks? We could of course include those in our github actions, but perhaps they aren't build by default?
   
   Another option is to put integrations in separate repos. That makes integration tests a bit more complicated. And I'm not sure how much benefit there is. The clear separate is maybe nice, but perhaps that's too much separation.
   
   Note that there also exists Daffodil NiFi processor: https://github.com/TresysTechnology/nifi-daffodil
   
   We probably want to determine how that might integrate in this as well. Perhaps it can all go in a single nar?
   
   Another option, maybe we should attempt to upstream this into NiFi and the only thing that exists in this repo are the abstract walker and the API changes (plus regression tests not specific to NiFi). There is already an open pull request to add the Nif Processor here: https://github.com/apache/nifi/pull/3130. The NiFi devs didn't seem adverse to Daffodil support, but wondering if Records is the better approch. So they may be open to accepting this kind of change.

##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.dsom.walker
+
+import java.io.File
+
+import org.apache.daffodil.compiler.Compiler
+import org.apache.daffodil.dsom.{Choice, ComplexTypeBase, ElementBase, GroupRef, ModelGroup, Root, Sequence, SimpleTypeBase, Term}
+
+/**
+ * A class designed to walk the internal representation of a DFDL Schema File.
+ *
+ * There are numerous internal event handlers to be defined by the implementing class; each
+ * corresponds to "encountering" a particular element type in the DFDL Schema file.
+ *
+ * Though recursion is used here to define the walk, it is not advised to use recursion between
+ * these event handlers.  Instead, consider a stack-like structure, as the DFDL Schema structure
+ * as well as the recursive method call structure can be represented by trees.
+ * @param schemaFile an input DFDL Schema file to parse
+ * @tparam T the return type of walkDSOMSchema and onWalkEnd.  This should be a class used to
+ *           represent the structure of the Schema.  You could always make it Unit if you don't
+ *           want these methods to return anything.
+ */
+abstract class AbstractDSOMWalker[T](schemaFile: File) {
+  /**
+   * The root element of the DFDL Schema.  This will be the starting point for the traversal
+   */
+  private final val schemaSetRoot: Root = Compiler().compileFile(schemaFile).sset.root
+
+  /**
+   * Auxiliary constructor to initialize this class with a path to a schema File rather than
+   * a File object directly
+   * @param pathToSchemaFile path to the DFDL Schema file desired to be parsed
+   */
+  def this(pathToSchemaFile: String) = this(new File(pathToSchemaFile))
+
+  /**
+   * Method to be called on the beginning of the traversal.  It is recommended to add some
+   * sort of wrapper element to a stack if you're doing a typical stack-based traversal.
+   * @param root the root element of the DFDL Schema
+   */
+  protected def onWalkBegin(root: Root): Unit
+
+  /**
+   * Method to be called when the traversal concludes.  It is recommended to put any post-processing
+   * and anything to tidy up the stack or the result here.
+   * @param root the root element of the DFDL Schema
+   * @return the result of the traversal
+   */
+  protected def onWalkEnd(root: Root): T
+
+  /**
+   * Method to be called when a *sequence* element is encountered.
+   * This is not really your array type; the typical "array" you'll see in an infoset
+   * is a special kind of Element, so it will be encountered
+   * in onElementBegin
+   * @param sequenceElement the sequence element
+   */
+  protected def onSequenceBegin(sequenceElement: Sequence): Unit
+
+  /**
+   * Method to be called when all the children of a sequence element are done being processed
+   * @param sequenceElement the original containing sequence element
+   */
+  protected def onSequenceEnd(sequenceElement: Sequence): Unit
+
+  /**
+   * Method to be called when a *GroupRef* element is encountered.  In RecordWalker we will treat
+   * this exactly like it's a Sequence, but you have the option to treat it differently.
+   * in onElementBegin
+   * @param groupRef the GroupRef element
+   */
+  protected def onGroupRefBegin(groupRef: GroupRef): Unit
+
+  /**
+   * Method to be called when all the children of a GroupRef element are done being processed
+   * @param groupRef the original containing GroupRef element
+   */
+  protected def onGroupRefEnd(groupRef: GroupRef): Unit
+
+  /**
+   * Method to be called when a *choice* element is encountered.
+   * A choice element is an important type and has significant
+   * implications for the resulting infoset.
+   * @param choiceElement the choice element that was encountered
+   */
+  protected def onChoiceBegin(choiceElement: Choice): Unit
+
+  /**
+   * Method to be called when all the elements within a choice element have been processed.
+   * @param choiceElement the original containing choice element
+   */
+  protected def onChoiceEnd(choiceElement: Choice): Unit
+
+  /**
+   * Method to be called when a "simple" element is encountered.  This isn't really used
+   * in the RecordWalker implementation since it's usually just a wrapper for an element
+   * with the simple you really care about, but it is here just in case.
+   * @param simpleElement the simple element that was encountered
+   */
+  protected def onSimpleBegin(simpleElement: SimpleTypeBase): Unit
+  /**
+   * Method to be called after a "simple" element has been processed.
+   * @param simpleElement the simple element that was encountered
+   */
+  protected def onSimpleEnd(simpleElement: SimpleTypeBase): Unit
+
+  /**
+   * Method to be called when a "complex" element is encountered.  This isn't really used
+   * in the RecordWalker implementation since it's usually just a wrapper for some inner element.
+   * @param complexElement the complex element that was encountered
+   */
+  protected def onComplexBegin(complexElement: ComplexTypeBase): Unit
+  /**
+   * Method to be called after a "complex" element has been processed.
+   * @param complexElement the complex element that was encountered
+   */
+  protected def onComplexEnd(complexElement: ComplexTypeBase): Unit
+
+  /**
+   * Method to be called when a regular element is encountered.  This is one of the most
+   * important methods; most DFDL elements with useful data become some subclass of
+   * ElementBase.
+   * @param element the element that was encountered
+   */
+  protected def onElementBegin(element: ElementBase): Unit
+  /**
+   * Method to be called when a regular element has been processed.
+   * @param element the element that was encountered
+   */
+  protected def onElementEnd(element: ElementBase): Unit
+
+  final def walkDSOMSchema(): T = {
+    onWalkBegin(schemaSetRoot)
+    // this is allowed because Root is a subclass of ElementBase
+    walkerHelper(schemaSetRoot)
+    // onWalkEnd will return the value returned by this method!
+    onWalkEnd(schemaSetRoot)
+  }
+
+  /**
+   * Overloaded walkerHelper method used for when some "group" element is encountered in
+   * the DSOM; the group elements are Sequences, Choices, and GroupRefs
+   * @param modelGroup the ModelGroup element that was encountered in the Schema
+   */
+  private final def walkerHelper(modelGroup: ModelGroup): Unit = {
+    modelGroup match {
+      case sequence: Sequence =>
+        onSequenceBegin(sequence)
+        for (child <- sequence.groupMembers) walkerHelper(child)

Review comment:
       I think the more scala-y way to do this is ``sequence.groupMembers.foreach(walkerHelper)``

##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.dsom.walker
+
+import java.io.File
+
+import org.apache.daffodil.compiler.Compiler
+import org.apache.daffodil.dsom.{Choice, ComplexTypeBase, ElementBase, GroupRef, ModelGroup, Root, Sequence, SimpleTypeBase, Term}
+
+/**
+ * A class designed to walk the internal representation of a DFDL Schema File.
+ *
+ * There are numerous internal event handlers to be defined by the implementing class; each
+ * corresponds to "encountering" a particular element type in the DFDL Schema file.
+ *
+ * Though recursion is used here to define the walk, it is not advised to use recursion between
+ * these event handlers.  Instead, consider a stack-like structure, as the DFDL Schema structure
+ * as well as the recursive method call structure can be represented by trees.
+ * @param schemaFile an input DFDL Schema file to parse
+ * @tparam T the return type of walkDSOMSchema and onWalkEnd.  This should be a class used to
+ *           represent the structure of the Schema.  You could always make it Unit if you don't
+ *           want these methods to return anything.
+ */
+abstract class AbstractDSOMWalker[T](schemaFile: File) {
+  /**
+   * The root element of the DFDL Schema.  This will be the starting point for the traversal
+   */
+  private final val schemaSetRoot: Root = Compiler().compileFile(schemaFile).sset.root

Review comment:
       What if the schema compilation fails? This doesn't really provide a way to handle that.
   
   I'm wondering if perhaps rather than creating a walker and having it compile things behind the scenes, the API actually becomes part of the DSOM components? For example:
   ```scala
   val c = Daffodil.compiler()
   val pf = c.compileFile(schemaFile)
   if (pf.isError) ...
   val dp = pf.onPath("/")
   if (dp.isError) ...
   // The above is all stanard Daffodil API schema compilation
   // Since that all succeeded, lets walk the dsom
   val walker = new MyCustomDsomWalker()
   pf.walkDSOMSchema(walker)
   ```
   So it makes the user responsible for compiling the schema and checking for error, which is normal and allows the user to check for schema compilation errors. If that all succeeds, then they can walk the DSOM. Then your walkDSOMSchema moves to the ProcessorFactor and it becomes the entrypoint.
   

##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.dsom.walker
+
+import java.io.File
+
+import org.apache.daffodil.compiler.Compiler
+import org.apache.daffodil.dsom.{Choice, ComplexTypeBase, ElementBase, GroupRef, ModelGroup, Root, Sequence, SimpleTypeBase, Term}
+
+/**
+ * A class designed to walk the internal representation of a DFDL Schema File.
+ *
+ * There are numerous internal event handlers to be defined by the implementing class; each
+ * corresponds to "encountering" a particular element type in the DFDL Schema file.
+ *
+ * Though recursion is used here to define the walk, it is not advised to use recursion between
+ * these event handlers.  Instead, consider a stack-like structure, as the DFDL Schema structure
+ * as well as the recursive method call structure can be represented by trees.
+ * @param schemaFile an input DFDL Schema file to parse
+ * @tparam T the return type of walkDSOMSchema and onWalkEnd.  This should be a class used to
+ *           represent the structure of the Schema.  You could always make it Unit if you don't
+ *           want these methods to return anything.
+ */
+abstract class AbstractDSOMWalker[T](schemaFile: File) {
+  /**
+   * The root element of the DFDL Schema.  This will be the starting point for the traversal
+   */
+  private final val schemaSetRoot: Root = Compiler().compileFile(schemaFile).sset.root
+
+  /**
+   * Auxiliary constructor to initialize this class with a path to a schema File rather than
+   * a File object directly
+   * @param pathToSchemaFile path to the DFDL Schema file desired to be parsed
+   */
+  def this(pathToSchemaFile: String) = this(new File(pathToSchemaFile))
+
+  /**
+   * Method to be called on the beginning of the traversal.  It is recommended to add some
+   * sort of wrapper element to a stack if you're doing a typical stack-based traversal.
+   * @param root the root element of the DFDL Schema
+   */
+  protected def onWalkBegin(root: Root): Unit
+
+  /**
+   * Method to be called when the traversal concludes.  It is recommended to put any post-processing
+   * and anything to tidy up the stack or the result here.
+   * @param root the root element of the DFDL Schema
+   * @return the result of the traversal
+   */
+  protected def onWalkEnd(root: Root): T
+
+  /**
+   * Method to be called when a *sequence* element is encountered.
+   * This is not really your array type; the typical "array" you'll see in an infoset
+   * is a special kind of Element, so it will be encountered
+   * in onElementBegin
+   * @param sequenceElement the sequence element
+   */
+  protected def onSequenceBegin(sequenceElement: Sequence): Unit

Review comment:
       I'm a little hesitant to give a walker implementation direct access to all of our internal schema compontents. It makes it much more difficult to make changes to these SchemaComponents since we need to be concerned about backwards compatibilty. We sort of have that problem with our InfosetOutputters, I'd like to not repeat that mistake.
   
   I'm wondering if it would be possible to come up with a reasonable list of properties that DSOM walkers actually need and only make those avaialble? I suspect this would be a pretty small list llike namespace, name, type, min/maxOccurs, and maybe a couple more properties?
   
   Perhaps your SchemaNode is the correct abstraction? Maybe fore each walker event, the we should be creating a SchemaNode and passing that in? Then we only have to worry about backwards compatibility with SchemaNodes, which should be significantly easier than our SchemaComponents?

##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.dsom.walker
+
+import java.io.File
+
+import org.apache.daffodil.compiler.Compiler
+import org.apache.daffodil.dsom.{Choice, ComplexTypeBase, ElementBase, GroupRef, ModelGroup, Root, Sequence, SimpleTypeBase, Term}
+
+/**
+ * A class designed to walk the internal representation of a DFDL Schema File.
+ *
+ * There are numerous internal event handlers to be defined by the implementing class; each
+ * corresponds to "encountering" a particular element type in the DFDL Schema file.
+ *
+ * Though recursion is used here to define the walk, it is not advised to use recursion between
+ * these event handlers.  Instead, consider a stack-like structure, as the DFDL Schema structure
+ * as well as the recursive method call structure can be represented by trees.
+ * @param schemaFile an input DFDL Schema file to parse
+ * @tparam T the return type of walkDSOMSchema and onWalkEnd.  This should be a class used to
+ *           represent the structure of the Schema.  You could always make it Unit if you don't
+ *           want these methods to return anything.
+ */
+abstract class AbstractDSOMWalker[T](schemaFile: File) {
+  /**
+   * The root element of the DFDL Schema.  This will be the starting point for the traversal
+   */
+  private final val schemaSetRoot: Root = Compiler().compileFile(schemaFile).sset.root
+
+  /**
+   * Auxiliary constructor to initialize this class with a path to a schema File rather than
+   * a File object directly
+   * @param pathToSchemaFile path to the DFDL Schema file desired to be parsed
+   */
+  def this(pathToSchemaFile: String) = this(new File(pathToSchemaFile))
+
+  /**
+   * Method to be called on the beginning of the traversal.  It is recommended to add some
+   * sort of wrapper element to a stack if you're doing a typical stack-based traversal.
+   * @param root the root element of the DFDL Schema
+   */
+  protected def onWalkBegin(root: Root): Unit
+
+  /**
+   * Method to be called when the traversal concludes.  It is recommended to put any post-processing
+   * and anything to tidy up the stack or the result here.
+   * @param root the root element of the DFDL Schema
+   * @return the result of the traversal
+   */
+  protected def onWalkEnd(root: Root): T
+
+  /**
+   * Method to be called when a *sequence* element is encountered.
+   * This is not really your array type; the typical "array" you'll see in an infoset
+   * is a special kind of Element, so it will be encountered
+   * in onElementBegin
+   * @param sequenceElement the sequence element
+   */
+  protected def onSequenceBegin(sequenceElement: Sequence): Unit
+
+  /**
+   * Method to be called when all the children of a sequence element are done being processed
+   * @param sequenceElement the original containing sequence element
+   */
+  protected def onSequenceEnd(sequenceElement: Sequence): Unit
+
+  /**
+   * Method to be called when a *GroupRef* element is encountered.  In RecordWalker we will treat
+   * this exactly like it's a Sequence, but you have the option to treat it differently.
+   * in onElementBegin
+   * @param groupRef the GroupRef element
+   */
+  protected def onGroupRefBegin(groupRef: GroupRef): Unit
+
+  /**
+   * Method to be called when all the children of a GroupRef element are done being processed
+   * @param groupRef the original containing GroupRef element
+   */
+  protected def onGroupRefEnd(groupRef: GroupRef): Unit
+
+  /**
+   * Method to be called when a *choice* element is encountered.
+   * A choice element is an important type and has significant
+   * implications for the resulting infoset.
+   * @param choiceElement the choice element that was encountered
+   */
+  protected def onChoiceBegin(choiceElement: Choice): Unit
+
+  /**
+   * Method to be called when all the elements within a choice element have been processed.
+   * @param choiceElement the original containing choice element
+   */
+  protected def onChoiceEnd(choiceElement: Choice): Unit
+
+  /**
+   * Method to be called when a "simple" element is encountered.  This isn't really used
+   * in the RecordWalker implementation since it's usually just a wrapper for an element
+   * with the simple you really care about, but it is here just in case.
+   * @param simpleElement the simple element that was encountered
+   */
+  protected def onSimpleBegin(simpleElement: SimpleTypeBase): Unit
+  /**
+   * Method to be called after a "simple" element has been processed.
+   * @param simpleElement the simple element that was encountered
+   */
+  protected def onSimpleEnd(simpleElement: SimpleTypeBase): Unit
+
+  /**
+   * Method to be called when a "complex" element is encountered.  This isn't really used
+   * in the RecordWalker implementation since it's usually just a wrapper for some inner element.
+   * @param complexElement the complex element that was encountered
+   */
+  protected def onComplexBegin(complexElement: ComplexTypeBase): Unit
+  /**
+   * Method to be called after a "complex" element has been processed.
+   * @param complexElement the complex element that was encountered
+   */
+  protected def onComplexEnd(complexElement: ComplexTypeBase): Unit
+
+  /**
+   * Method to be called when a regular element is encountered.  This is one of the most
+   * important methods; most DFDL elements with useful data become some subclass of
+   * ElementBase.
+   * @param element the element that was encountered
+   */
+  protected def onElementBegin(element: ElementBase): Unit
+  /**
+   * Method to be called when a regular element has been processed.
+   * @param element the element that was encountered
+   */
+  protected def onElementEnd(element: ElementBase): Unit
+
+  final def walkDSOMSchema(): T = {
+    onWalkBegin(schemaSetRoot)
+    // this is allowed because Root is a subclass of ElementBase
+    walkerHelper(schemaSetRoot)
+    // onWalkEnd will return the value returned by this method!
+    onWalkEnd(schemaSetRoot)
+  }
+
+  /**
+   * Overloaded walkerHelper method used for when some "group" element is encountered in
+   * the DSOM; the group elements are Sequences, Choices, and GroupRefs
+   * @param modelGroup the ModelGroup element that was encountered in the Schema
+   */
+  private final def walkerHelper(modelGroup: ModelGroup): Unit = {
+    modelGroup match {
+      case sequence: Sequence =>
+        onSequenceBegin(sequence)
+        for (child <- sequence.groupMembers) walkerHelper(child)
+        onSequenceEnd(sequence)
+      case choice: Choice =>
+        onChoiceBegin(choice)
+        for (child <- choice.groupMembers) walkerHelper(child)
+        onChoiceEnd(choice)
+      case groupRef: GroupRef =>
+        if (!groupRef.isHidden) {
+          onGroupRefBegin(groupRef)
+          for (child <- groupRef.groupMembers) walkerHelper(child)
+          onGroupRefEnd(groupRef)
+        }
+    }
+  }
+
+  /**
+   * Overloaded walkerHelper method that will handle the input either as an ElementBase or ModelGroup
+   * @param term the Term element that was encountered in the Schema
+   */
+  private final def walkerHelper(term: Term): Unit = {
+    term match {
+      case element: ElementBase => walkerHelper(element)
+      case modelGroup: ModelGroup => walkerHelper(modelGroup)
+    }
+  }
+
+  /**
+   * Overloaded walkerHelper method that will handle some element either as a complex or simple type
+   * @param element the element that was encountered in the Schema
+   */
+  private final def walkerHelper(element: ElementBase): Unit = {
+    onElementBegin(element)
+    if (element.isComplexType) {
+      onComplexBegin(element.complexType)
+      walkerHelper(element.complexType.group)
+      onComplexEnd(element.complexType)
+    } else {
+      onSimpleBegin(element.simpleType)
+      onSimpleEnd(element.simpleType)
+    }
+    onElementEnd(element)
+  }
+}

Review comment:
       I'm not sure much is gained by spliting the walkerHelper's into different overloaded methods. Just sort of adds another layer of indirection. I think I'd rather see it as one function that accepts a Term and is something like:
   ```scala
   term match {
    case s: Sequence => ...
    case c: Choice => ...
    case g: GroupRef => ...
    case: e: ElementBase => ...
   }
   ```
   Another option, taking my first comment a  little further, perhaps instead of the walker having the logic for walking, it all moves into the DSOM. We require that each SchemaComponent implement a ``def walkDSOM(w: Walker)`` method. So for example:
   ```scala
   class Seqeunce {
     ...
     override def walkDSOM(w: Walker): Unit = {
       w.onSequenceBegin(...)
       groupMembers.forEach(_.walkDSOM(w))
       w.onSequenceEnd(...)
     }
   ```
   It's sortof two sides of the same coin, but this way the DSOM elements are more in reponsible for knowing how to walk their own DSOM rather than the walker having to know how to walk everything DSOM. And the AbstractWalker essentialy just becomes a WalkerHandler of events, and doesn't do any actual walking itself.

##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.dsom.walker
+
+import java.io.File
+
+import org.apache.daffodil.compiler.Compiler
+import org.apache.daffodil.dsom.{Choice, ComplexTypeBase, ElementBase, GroupRef, ModelGroup, Root, Sequence, SimpleTypeBase, Term}
+
+/**
+ * A class designed to walk the internal representation of a DFDL Schema File.
+ *
+ * There are numerous internal event handlers to be defined by the implementing class; each
+ * corresponds to "encountering" a particular element type in the DFDL Schema file.
+ *
+ * Though recursion is used here to define the walk, it is not advised to use recursion between
+ * these event handlers.  Instead, consider a stack-like structure, as the DFDL Schema structure
+ * as well as the recursive method call structure can be represented by trees.
+ * @param schemaFile an input DFDL Schema file to parse
+ * @tparam T the return type of walkDSOMSchema and onWalkEnd.  This should be a class used to
+ *           represent the structure of the Schema.  You could always make it Unit if you don't
+ *           want these methods to return anything.
+ */
+abstract class AbstractDSOMWalker[T](schemaFile: File) {
+  /**
+   * The root element of the DFDL Schema.  This will be the starting point for the traversal
+   */
+  private final val schemaSetRoot: Root = Compiler().compileFile(schemaFile).sset.root
+
+  /**
+   * Auxiliary constructor to initialize this class with a path to a schema File rather than
+   * a File object directly
+   * @param pathToSchemaFile path to the DFDL Schema file desired to be parsed
+   */
+  def this(pathToSchemaFile: String) = this(new File(pathToSchemaFile))
+
+  /**
+   * Method to be called on the beginning of the traversal.  It is recommended to add some
+   * sort of wrapper element to a stack if you're doing a typical stack-based traversal.
+   * @param root the root element of the DFDL Schema
+   */
+  protected def onWalkBegin(root: Root): Unit
+
+  /**
+   * Method to be called when the traversal concludes.  It is recommended to put any post-processing
+   * and anything to tidy up the stack or the result here.
+   * @param root the root element of the DFDL Schema
+   * @return the result of the traversal
+   */
+  protected def onWalkEnd(root: Root): T
+
+  /**
+   * Method to be called when a *sequence* element is encountered.
+   * This is not really your array type; the typical "array" you'll see in an infoset
+   * is a special kind of Element, so it will be encountered
+   * in onElementBegin
+   * @param sequenceElement the sequence element
+   */
+  protected def onSequenceBegin(sequenceElement: Sequence): Unit
+
+  /**
+   * Method to be called when all the children of a sequence element are done being processed
+   * @param sequenceElement the original containing sequence element
+   */
+  protected def onSequenceEnd(sequenceElement: Sequence): Unit
+
+  /**
+   * Method to be called when a *GroupRef* element is encountered.  In RecordWalker we will treat
+   * this exactly like it's a Sequence, but you have the option to treat it differently.
+   * in onElementBegin
+   * @param groupRef the GroupRef element
+   */
+  protected def onGroupRefBegin(groupRef: GroupRef): Unit
+
+  /**
+   * Method to be called when all the children of a GroupRef element are done being processed
+   * @param groupRef the original containing GroupRef element
+   */
+  protected def onGroupRefEnd(groupRef: GroupRef): Unit
+
+  /**
+   * Method to be called when a *choice* element is encountered.
+   * A choice element is an important type and has significant
+   * implications for the resulting infoset.
+   * @param choiceElement the choice element that was encountered
+   */
+  protected def onChoiceBegin(choiceElement: Choice): Unit
+
+  /**
+   * Method to be called when all the elements within a choice element have been processed.
+   * @param choiceElement the original containing choice element
+   */
+  protected def onChoiceEnd(choiceElement: Choice): Unit
+
+  /**
+   * Method to be called when a "simple" element is encountered.  This isn't really used
+   * in the RecordWalker implementation since it's usually just a wrapper for an element
+   * with the simple you really care about, but it is here just in case.
+   * @param simpleElement the simple element that was encountered
+   */
+  protected def onSimpleBegin(simpleElement: SimpleTypeBase): Unit
+  /**
+   * Method to be called after a "simple" element has been processed.
+   * @param simpleElement the simple element that was encountered
+   */
+  protected def onSimpleEnd(simpleElement: SimpleTypeBase): Unit
+
+  /**
+   * Method to be called when a "complex" element is encountered.  This isn't really used
+   * in the RecordWalker implementation since it's usually just a wrapper for some inner element.
+   * @param complexElement the complex element that was encountered
+   */
+  protected def onComplexBegin(complexElement: ComplexTypeBase): Unit
+  /**
+   * Method to be called after a "complex" element has been processed.
+   * @param complexElement the complex element that was encountered
+   */
+  protected def onComplexEnd(complexElement: ComplexTypeBase): Unit
+
+  /**
+   * Method to be called when a regular element is encountered.  This is one of the most
+   * important methods; most DFDL elements with useful data become some subclass of
+   * ElementBase.
+   * @param element the element that was encountered
+   */
+  protected def onElementBegin(element: ElementBase): Unit
+  /**
+   * Method to be called when a regular element has been processed.
+   * @param element the element that was encountered
+   */
+  protected def onElementEnd(element: ElementBase): Unit
+
+  final def walkDSOMSchema(): T = {
+    onWalkBegin(schemaSetRoot)
+    // this is allowed because Root is a subclass of ElementBase
+    walkerHelper(schemaSetRoot)
+    // onWalkEnd will return the value returned by this method!
+    onWalkEnd(schemaSetRoot)
+  }
+
+  /**
+   * Overloaded walkerHelper method used for when some "group" element is encountered in
+   * the DSOM; the group elements are Sequences, Choices, and GroupRefs
+   * @param modelGroup the ModelGroup element that was encountered in the Schema
+   */
+  private final def walkerHelper(modelGroup: ModelGroup): Unit = {
+    modelGroup match {
+      case sequence: Sequence =>
+        onSequenceBegin(sequence)
+        for (child <- sequence.groupMembers) walkerHelper(child)
+        onSequenceEnd(sequence)
+      case choice: Choice =>
+        onChoiceBegin(choice)
+        for (child <- choice.groupMembers) walkerHelper(child)
+        onChoiceEnd(choice)
+      case groupRef: GroupRef =>
+        if (!groupRef.isHidden) {
+          onGroupRefBegin(groupRef)
+          for (child <- groupRef.groupMembers) walkerHelper(child)
+          onGroupRefEnd(groupRef)
+        }
+    }
+  }
+
+  /**
+   * Overloaded walkerHelper method that will handle the input either as an ElementBase or ModelGroup
+   * @param term the Term element that was encountered in the Schema
+   */
+  private final def walkerHelper(term: Term): Unit = {
+    term match {
+      case element: ElementBase => walkerHelper(element)
+      case modelGroup: ModelGroup => walkerHelper(modelGroup)
+    }
+  }
+
+  /**
+   * Overloaded walkerHelper method that will handle some element either as a complex or simple type
+   * @param element the element that was encountered in the Schema
+   */
+  private final def walkerHelper(element: ElementBase): Unit = {
+    onElementBegin(element)
+    if (element.isComplexType) {
+      onComplexBegin(element.complexType)
+      walkerHelper(element.complexType.group)
+      onComplexEnd(element.complexType)
+    } else {
+      onSimpleBegin(element.simpleType)
+      onSimpleEnd(element.simpleType)
+    }
+    onElementEnd(element)
+  }
+}
+
+/**
+ * The sole purpose of this Companion Object is to contain this one static variable.
+ */
+object AbstractDSOMWalker {
+  /**
+   * Used to suppress all of the extra printing from running test cases
+   */
+  final val PRODUCTION_MODE: Boolean = true
+}

Review comment:
       This is only used in the NiFI specific Record stuff. I would expect this to just be a flag in the RecordWalker. Presumable it would plug into NiFi's logging mechanism.




----------------------------------------------------------------
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