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/08 17:50:58 UTC

[GitHub] [incubator-daffodil] andrewjc2000 opened a new pull request #394: Feature Addition: API for traversing DSOM and generating NIFI RecordSchemas from DSOM

andrewjc2000 opened a new pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394


   In this fork, no code was modified other than 2 added dependencies, and several classes to define an API and test them were added.  These were all in the org.apache.daffodil.dsom package.  Here is a summary of the additions:
   - An abstract class called AbstractDSOMWalker that traverses DSOM objects. As the DSOM is traversed, various event handlers that are declared abstract in the API are invoked, depending on the DSOM element encountered (Sequence, Element, Choice, etc.).
   - A concrete implementation of AbstractDSOMWalker called RecordWalker that generates a NIFI RecordSchema as a result of the DSOM traversal.
   - A helper class called SchemaNode for storing the relevant information about an element in the DSOM.  These are used in the intermediate stages of processing in RecordWalker.
   - 3 Test classes and 3 complete DFDL Schema files for testing the output of RecordWalker.  These were written in Scalatest.
   
   As a result of the above features, 2 new dependencies were also added for Core in project/Dependencies: one for NIFI Records, and another for ScalaTest


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



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

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455804349



##########
File path: daffodil-japi/src/main/scala/org/apache/daffodil/japi/Daffodil.scala
##########
@@ -373,6 +374,12 @@ class ProcessorFactory private[japi] (private var pf: SProcessorFactory)
     new DataProcessor(dp)
   }
 
+  /**
+   * Exposes the RootView object corresponding to this ProcessorFactory.  This can
+   * be used to start a walk using the walkFromRoot method in a DSOM Walker.
+   */
+  def rootView: RootView = pf.rootView

Review comment:
       We probably should document that this is an experiemental API and is subject to change. Once we stabalize this and confirm we have the right functionality, we can remove the experiemtal tag and make the Views part of the public API.




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r454720851



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,163 @@
+/*
+ * 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 org.apache.daffodil.dpath.NodeInfo.PrimType
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+  def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {

Review comment:
       This would make the API larger due to this extra flexibility.  I went ahead and removed the `final` modifier from this method in the latest revision.




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455969304



##########
File path: daffodil-core/src/test/scala/org/apache/daffodil/dsom/walker/BasicWalker.scala
##########
@@ -0,0 +1,50 @@
+/*
+ * 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 scala.collection.mutable.ArrayBuffer
+
+class BasicWalker(ignoreTypeWrappers: Boolean = false, onlyElements: Boolean = false) extends AbstractDSOMWalker {
+
+  var nodeArr: ArrayBuffer[Either[TermView, TypeView]] = ArrayBuffer()
+
+  override protected def onWalkBegin(root: RootView): Unit = {}
+
+  override protected def onWalkEnd(root: RootView): Unit = {}
+
+  override def onTermBegin(termElement: TermView): Unit = {
+    if (onlyElements) {
+      termElement match {
+        case _: ElementBaseView => nodeArr += Left(termElement)
+        case _ =>
+      }
+    } else {
+      nodeArr += Left(termElement)
+    }
+  }

Review comment:
       Yes, that is much cleaner - I implemented this along with another case statement based on a new boolean for the tester class in the latest revision.




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r454728698



##########
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:
       Now that the walk is initiated externally from `ProcessorFactory` rather than from the walker itself, there is currently no way to start it via the Java or Scala API.  I am going to go ahead and add the `walkDSOM` method to the `ProcessorFactory` in the JAPI and SAPI modules; let me know if you think something else should be done instead.




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455879433



##########
File path: daffodil-core/src/test/scala/org/apache/daffodil/dsom/walker/BasicWalker.scala
##########
@@ -0,0 +1,50 @@
+/*
+ * 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 scala.collection.mutable.ArrayBuffer
+
+class BasicWalker(ignoreTypeWrappers: Boolean = false, onlyElements: Boolean = false) extends AbstractDSOMWalker {
+
+  var nodeArr: ArrayBuffer[Either[TermView, TypeView]] = ArrayBuffer()
+
+  override protected def onWalkBegin(root: RootView): Unit = {}
+
+  override protected def onWalkEnd(root: RootView): Unit = {}
+
+  override def onTermBegin(termElement: TermView): Unit = {
+    if (onlyElements) {
+      termElement match {
+        case _: ElementBaseView => nodeArr += Left(termElement)
+        case _ =>
+      }
+    } else {
+      nodeArr += Left(termElement)
+    }
+  }
+
+  override def onTermEnd(termElement: TermView): Unit = {}
+
+  override def onTypeBegin(typeElement: TypeView): Unit = {
+    if (!ignoreTypeWrappers) {
+      nodeArr += Right(typeElement)
+    }
+  }
+
+  override def onTypeEnd(typeElement: TypeView): Unit = {}
+}

Review comment:
       It ignores all end events because I wanted the end result to be a stack of every element encountered rather than a stack of Nodes with child elements.  In all the implementations so far, End events were just used to remove the current element from the stack, which I don't want here.




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



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

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#issuecomment-659669263


   @andrewjc2000, with two +1's this is ready to merge. Can you please squash all of your changes into a single commit, update the commit message based on the squashed state of the code, and add the bug number to the end of the commit message? Then this will be good to merge. Thanks for the contribution!


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



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

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r454298618



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/SimpleTypes.scala
##########
@@ -53,9 +54,9 @@ trait TypeBase {
 trait NonPrimTypeMixin
 
 sealed trait SimpleTypeBase extends TypeBase
-  with HasOptRepTypeMixin {
+  with HasOptRepTypeMixin with SimpleTypeView {
 
-  def primType: PrimType
+  override def primType: PrimType

Review comment:
       Doesn't seem unreasonable. I had forgotten that NodeInfo is defined in daffodil-runtime1. The problem is daffodil-core depends on daffodil-runtime1, (core creates many of the objects defined in runtime1), so runtime1 can't use anything defined in core or we'd have a circular dependency. They way you handled it seems reasonable.
   
   If you wanted to do it with mixin views, you would need to define the PrimTypeView traits in the daffodil-runtime1 class. They can still be in a dsom.walker package, just in a different subproject/jar.




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453127981



##########
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:
       This all makes a lot more sense idiomatically.  In the latest commit I opted for adding this method in ProcessorFactory:
   `def walkDSOM[T](walker: AbstractDSOMWalker[T]): T = walker.walkFromRoot(sset.root)`
   AbstractDSOMWalker now also does not take in any constructor parameters.




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



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

Posted by GitBox <gi...@apache.org>.
mbeckerle commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453002919



##########
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:
       The abstract walker can define a set of view mixin traits that we mix into each walked object. The traits may be specific to each walked DSOM object as it has to expose different things from them perhaps.
   
   The walker API would be defined in terms of these traits. The traits would have public members that are the things we choose to formally expose (and support) on the walker API. 




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



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

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455765810



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,177 @@
+/*
+ * 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
+
+/**
+ * Below is a series of "View" Mixins defined such that particular simple attributes
+ * of the core DSOM classes are exposed as a Walk is performed.
+ *
+ * Several of them have a walkDSOM method.  By default, this method is called as the walk is performed,
+ * but a user implementing AbstractDSOMWalker can override this behavior by overriding walkFromRoot
+ * and implementing their own event handlers for each of the DSOM types.
+ */
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+}
+
+trait TypeView
+trait SimpleTypeView extends TypeView {
+  def primType: PrimTypeView
+}
+trait ComplexTypeView extends TypeView {
+  def group: ModelGroupView
+}
+
+trait ModelGroupView extends TermView {
+  def groupMembers: Seq[TermView]
+}
+
+trait ChoiceView extends ModelGroupView
+trait SequenceView extends ModelGroupView
+trait GroupRefView extends ModelGroupView {
+  def isHidden: Boolean
+}
+
+trait ElementDeclView extends CommonContextView {
+  def isSimpleType: Boolean
+  def isComplexType: Boolean
+  def simpleType: SimpleTypeView
+  def complexType: ComplexTypeView
+  def hasDefaultValue: Boolean
+}
+
+trait ElementBaseView extends ElementDeclView with TermView {
+  def name: String
+}
+
+trait RootView extends ElementBaseView
+
+/**
+ * A class designed to walk the internal representation of a DFDL Schema File.
+ *
+ * There are 2 main event handlers an implementing class has to worry about: one for Terms, and
+ * another for Types.  These are called as the DSOM is walked.
+ *
+ * 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.
+ */
+abstract class AbstractDSOMWalker {
+
+  /**
+   * 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: RootView): 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
+   */
+  protected def onWalkEnd(root: RootView): Unit
+
+  /**
+   * Method to be called whenever any element that is a Term is encountered.
+   * This applies to Sequence, Choice, GroupRef, ElementBase, etc.
+   *
+   * It is highly recommended that, when implementing this method, you pattern match
+   * some of these different sub-types at some point to handle each accordingly.
+   *
+   * @param termElement the term element
+   */
+  def onTermBegin(termElement: TermView): Unit
+
+  /**
+   * Method to be called when a Term element has finished processing
+   *
+   * @param termElement the term element
+   */
+  def onTermEnd(termElement: TermView): Unit
+
+  /**
+   * Method to be called when either a Simple or Complex Type element has been encountered.
+   * This is just the element for <simpleType> or <complexType>, so no implementation is necessary if you
+   * do not care about these wrapper portion; its children will be walked automatically.
+   *
+   * @param typeElement either a Simple or Complex type
+   */
+  def onTypeBegin(typeElement: TypeView): Unit
+
+  /**
+   * Method to be called when either a Simple or Complex Type element has been finished
+   * processing.  See onTypeBegin method description
+   *
+   * @param typeElement either a Simple or Complex type
+   */
+  def onTypeEnd(typeElement: TypeView): Unit
+
+  /**
+   * Starts a DSOM walk from the Root object.  By default, the walk is completed recursively in the helper
+   * method below, but a custom walk can be defined by overriding
+   * this method instead and calling local event handlers.
+   * @param schemaSetRoot The root element of the DFDL Schema.  This will be the starting point of the traversal
+   */
+  def walkFromRoot(schemaSetRoot: RootView): Unit = {
+    onWalkBegin(schemaSetRoot)
+    walkerHelper(schemaSetRoot)
+    // onWalkEnd will return the value returned by this method!
+    onWalkEnd(schemaSetRoot)
+  }
+
+  /**
+   * Local helper method used to recursively walk the DSOM.
+   * If a non-recursive walk or a more specific recursive walk is desired, one
+   * can override this method, or simply not use it and instead override walkFromRoot
+   * and have that overridden method reference custom helper methods.
+   * @param termView the current Term element to be pattern matched
+   */
+  protected def walkerHelper(termView: TermView): Unit = {
+    onTermBegin(termView)
+    termView match {
+      case element: ElementBaseView =>
+        if (element.isComplexType) {
+          onTypeBegin(element.complexType)
+          walkerHelper(element.complexType.group)
+          onTypeEnd(element.complexType)
+        } else {
+          onTypeBegin(element.simpleType)
+          onTypeEnd(element.simpleType)
+        }
+      case groupRef: GroupRefView =>
+        if (!groupRef.isHidden) {
+          groupRef.groupMembers.foreach(walkerHelper)
+        }
+      case modelGroup: ModelGroupView =>
+        modelGroup.groupMembers.foreach(walkerHelper)
+      case _ =>

Review comment:
       Does thid default case ever get hit? I think this is impossible?




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r456009313



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,174 @@
+/*
+ * 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
+
+/**
+ * Below is a series of "View" Mixins defined such that particular simple attributes
+ * of the core DSOM classes are exposed as a Walk is performed.
+ *
+ * Several of them have a walkDSOM method.  By default, this method is called as the walk is performed,
+ * but a user implementing AbstractDSOMWalker can override this behavior by overriding walkFromRoot
+ * and implementing their own event handlers for each of the DSOM types.
+ */
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+}
+
+trait TypeView
+trait SimpleTypeView extends TypeView {
+  def primType: PrimTypeView
+}
+trait ComplexTypeView extends TypeView {
+  def group: ModelGroupView
+}
+
+trait ModelGroupView extends TermView {
+  def groupMembers: Seq[TermView]
+}
+
+trait ChoiceView extends ModelGroupView
+trait SequenceView extends ModelGroupView
+trait GroupRefView extends ModelGroupView {
+  def isHidden: Boolean

Review comment:
       I think there is a solution to this, but it would break the current paradigm of every field in the `View`s referencing an existing field.  We could replace the existing `group` element in `ComplexTypeView` with something like `optVisibleGroup`, and then the implementation for `optVisibleGroup` would pattern-match `group`; if it is hidden, return `None`, otherwise return the `group` wrapped in an option.  Similarly, instead of `groupMembers` being in `TermView`, we would have `visisbleGroupMembers`, and in the implementation `visibleGroupMembers` would be a filtered list of the real `groupMembers`, excluding any `TermView` that is a `GroupRef` and is also hidden.
   
   However, my main concern with this is it would potentially cause more confusion, leading the end user to wonder why we're only allowed to see "visible" members, and might make traversal very annoying with having to constantly unpack Options.




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



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

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453884432



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/SimpleTypes.scala
##########
@@ -53,9 +54,9 @@ trait TypeBase {
 trait NonPrimTypeMixin
 
 sealed trait SimpleTypeBase extends TypeBase
-  with HasOptRepTypeMixin {
+  with HasOptRepTypeMixin with SimpleTypeView {
 
-  def primType: PrimType
+  override def primType: PrimType

Review comment:
       You could probably just use the same trait trick same as the Views (e.g.  trait that's like org.apache.daffodil.dsom.walker.IntegerView which extends dsom.walker.PrimTypeView) and then the dpath.PrimType objects extend the appropriate view. Maybe this is excessive though? I'm trying to imagine a clean API where the user only has access to things in dsom.walker.*  There's a lot of duplication though.




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



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

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455915070



##########
File path: daffodil-japi/src/main/scala/org/apache/daffodil/japi/Daffodil.scala
##########
@@ -373,6 +374,12 @@ class ProcessorFactory private[japi] (private var pf: SProcessorFactory)
     new DataProcessor(dp)
   }
 
+  /**
+   * Exposes the RootView object corresponding to this ProcessorFactory.  This can
+   * be used to start a walk using the walkFromRoot method in a DSOM Walker.
+   */
+  def rootView: RootView = pf.rootView

Review comment:
       I know the Scala language sometimes uses an ``experimental`` object that things go in, so we could do something like that:
   ```scala
   class ProcoessorFactory(pf: SProcessorFactory) {
     ...
     object experimental {
       def rootView: RootView = pf.rootView
     }
     ...
   }
   ```
   To access the rootView, one would do something like this:
   ```scala
   val rootView = pf.experimental.rootView
   ```
   That makes it very obvious when you are using something that is experimental and that the API might change in the future. If we do go this route, I wonder if it also make sense to create a new sbt subproject and move all the new View traits and AbstratDSOMWalker in an experimental package (eg. org.apache.daffodil.experimental.dsom.walker), similar to how we have the daffodil-udf package split out? And we can use the new unidoc stuff to include it in public API docs?
   
   Not sure how I feel about this though. Maybe it's too obtrusive?
   
   I think the other option is to just have a blurb like "This API is experimental is and subject to change". I don't know if there's an annotation or standard javadoc practice for this kind of thing.




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453889437



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/SimpleTypes.scala
##########
@@ -53,9 +54,9 @@ trait TypeBase {
 trait NonPrimTypeMixin
 
 sealed trait SimpleTypeBase extends TypeBase
-  with HasOptRepTypeMixin {
+  with HasOptRepTypeMixin with SimpleTypeView {
 
-  def primType: PrimType
+  override def primType: PrimType

Review comment:
       You mentioned that `PrimType` is changed frequently - would this list pose an issue because of that? i.e. wouldn't future developers have to go change this list of primitive Views every time `PrimType` is changed?




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



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

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455806735



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,177 @@
+/*
+ * 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
+
+/**
+ * Below is a series of "View" Mixins defined such that particular simple attributes
+ * of the core DSOM classes are exposed as a Walk is performed.
+ *
+ * Several of them have a walkDSOM method.  By default, this method is called as the walk is performed,
+ * but a user implementing AbstractDSOMWalker can override this behavior by overriding walkFromRoot
+ * and implementing their own event handlers for each of the DSOM types.
+ */
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+}
+
+trait TypeView
+trait SimpleTypeView extends TypeView {
+  def primType: PrimTypeView
+}
+trait ComplexTypeView extends TypeView {
+  def group: ModelGroupView
+}
+
+trait ModelGroupView extends TermView {
+  def groupMembers: Seq[TermView]
+}
+
+trait ChoiceView extends ModelGroupView
+trait SequenceView extends ModelGroupView
+trait GroupRefView extends ModelGroupView {
+  def isHidden: Boolean
+}
+
+trait ElementDeclView extends CommonContextView {
+  def isSimpleType: Boolean
+  def isComplexType: Boolean
+  def simpleType: SimpleTypeView
+  def complexType: ComplexTypeView
+  def hasDefaultValue: Boolean

Review comment:
       Do we need to provide a method to get the default value? I assume if something wants to know about default values it will also want to know what that default it?




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



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

Posted by GitBox <gi...@apache.org>.
mbeckerle commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453004186



##########
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'd suggest the names should end with "View" so ComplexTypeView would be the trait mixed into ComplexTypeBase giving the walker access to its public members, which would just be pass through. 
   
   To cheat users could always downcast to ComplexTypeBase, but I think that's glaring enough that we can live with it. 




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455986288



##########
File path: daffodil-japi/src/main/scala/org/apache/daffodil/japi/Daffodil.scala
##########
@@ -373,6 +374,12 @@ class ProcessorFactory private[japi] (private var pf: SProcessorFactory)
     new DataProcessor(dp)
   }
 
+  /**
+   * Exposes the RootView object corresponding to this ProcessorFactory.  This can
+   * be used to start a walk using the walkFromRoot method in a DSOM Walker.
+   */
+  def rootView: RootView = pf.rootView

Review comment:
       While this solution is great and simple for Scala, it doesn't seem to translate well to Java; `pf.experimental.rootView()` does not compile, and IntelliJ thinks that `ProcessorFactory.experimental$.MODULE$.rootView()` compiles but this does not work when you go to build it, either.  Could we just create a brief inner wrapper class here with its only member as `rootView` and then make `experimental` a member of `ProcessorFactory` of that wrapper type?




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r454722287



##########
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:
       It definitely makes sense to keep the API and any NiFi-related/dependent code separate; in the latest revision I removed RecordWalker and SchemaRecordNode from core and made some general classes for testing to replace them.  I think it would make the most sense to move both of these to the nifi-daffodil repo, but as it currently stands they're both written in Scala and I'm not sure if there's necessarily a 1-1 Java conversion.




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



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

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455011060



##########
File path: daffodil-core/src/test/resources/test/standard-xsd/bmp.xsd
##########
@@ -0,0 +1,656 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--

Review comment:
       We generally avoid putting large schemas for file types inside the daffodil repo. Instead, we create smaller schemas that test the cases we are concerned about.




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



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

Posted by GitBox <gi...@apache.org>.
mbeckerle commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455860248



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,177 @@
+/*
+ * 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
+
+/**
+ * Below is a series of "View" Mixins defined such that particular simple attributes
+ * of the core DSOM classes are exposed as a Walk is performed.
+ *
+ * Several of them have a walkDSOM method.  By default, this method is called as the walk is performed,
+ * but a user implementing AbstractDSOMWalker can override this behavior by overriding walkFromRoot
+ * and implementing their own event handlers for each of the DSOM types.
+ */
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+}
+
+trait TypeView
+trait SimpleTypeView extends TypeView {
+  def primType: PrimTypeView
+}
+trait ComplexTypeView extends TypeView {
+  def group: ModelGroupView
+}
+
+trait ModelGroupView extends TermView {
+  def groupMembers: Seq[TermView]
+}
+
+trait ChoiceView extends ModelGroupView
+trait SequenceView extends ModelGroupView
+trait GroupRefView extends ModelGroupView {
+  def isHidden: Boolean
+}
+
+trait ElementDeclView extends CommonContextView {
+  def isSimpleType: Boolean
+  def isComplexType: Boolean
+  def simpleType: SimpleTypeView
+  def complexType: ComplexTypeView
+  def hasDefaultValue: Boolean

Review comment:
       maybe not? Why do we need to know if something has a default value even? Isn't that a format issue? Maybe we need it to point out an error where a required field can be absent from the NiFi record when unparsing it? But I don't actually see why we need the value. 




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



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

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455990094



##########
File path: daffodil-japi/src/main/scala/org/apache/daffodil/japi/Daffodil.scala
##########
@@ -373,6 +374,12 @@ class ProcessorFactory private[japi] (private var pf: SProcessorFactory)
     new DataProcessor(dp)
   }
 
+  /**
+   * Exposes the RootView object corresponding to this ProcessorFactory.  This can
+   * be used to start a walk using the walkFromRoot method in a DSOM Walker.
+   */
+  def rootView: RootView = pf.rootView

Review comment:
       Seems like a reasonable approach to me.




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453129119



##########
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:
       This all makes a lot of sense, hopefully I took care of it well in the latest version.
   
   I wound up defining a series of traits ending in View as Mike suggested, such as TermView and SequenceView.  Everything in RecordWalker and the helper SchemaNode (which I renamed to RecordSchemaNode to avoid confusion, it should _only_ be used for the NIFI RecordSchema, as it has an attribute that is evaluated to a NIFI Record Simple type, now uses all of these traits.
   
   I then went ahead and mixed in each of these traits into all of the appropriate existing DSOM objects.  Each attribute defined by the new Mixins matches an existing property in each of the classes, so it was not much of a hassle to add.  However, it was a bit difficult to get the inheritance structure right with respect to the mixins themselves (ElementBaseView mixes in both ElementDeclView and TermView), so please let me know if something seems off about it.
   
   The only "internal" item the traits can access that isn't a primitive or a String is the Enum PrimType class; I decided to keep it since it was highly useful when it came to simple types.  I hope leaving this in won't cause any issues.
   
   Within these traits I also defined a new "walkDSOM" method; hence moving all of those walkerHelper methods out of the abstract class and into the new traits.




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



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

Posted by GitBox <gi...@apache.org>.
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?




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



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

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455925365



##########
File path: daffodil-japi/src/main/scala/org/apache/daffodil/japi/Daffodil.scala
##########
@@ -373,6 +374,12 @@ class ProcessorFactory private[japi] (private var pf: SProcessorFactory)
     new DataProcessor(dp)
   }
 
+  /**
+   * Exposes the RootView object corresponding to this ProcessorFactory.  This can
+   * be used to start a walk using the walkFromRoot method in a DSOM Walker.
+   */
+  def rootView: RootView = pf.rootView

Review comment:
       Yeah, the submodule is really only needed when we want to include View stuff in public docs. It does mean in order to use the walker/views you kindof have to know what you're doing and know where to find these views. But that's not unreasonable for experimental things. Agree that the subprojet probably isn't needed right now.




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455887835



##########
File path: daffodil-core/src/test/scala/org/apache/daffodil/dsom/walker/TestDSOMWalker.scala
##########
@@ -0,0 +1,229 @@
+/*
+ * 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 org.apache.daffodil.util._
+import org.apache.daffodil.compiler.{ Compiler, ProcessorFactory }
+import org.junit.Test
+import org.junit.Assert._
+
+class TestDSOMWalker {
+
+  @Test def testComplexTypes(): Unit = {
+    val testSchema = SchemaUtils.dfdlTestSchema(
+      <xs:include schemaLocation="org/apache/daffodil/xsd/DFDLGeneralFormat.dfdl.xsd"/>,
+      <dfdl:format ref="ex:GeneralFormat"
+                   alignment="implicit" alignmentUnits="bits" occursCountKind="implicit"
+                   lengthKind="delimited" encoding="ASCII"/>,
+      <xs:element name="PersonData">
+        <xs:complexType>
+          <xs:sequence>
+            <xs:choice>
+              <xs:element name="age" type="xs:int" minOccurs="1" maxOccurs="1"/>
+            </xs:choice>
+            <xs:group ref="testGroup" />
+          </xs:sequence>
+        </xs:complexType>
+      </xs:element>
+      <xs:group name="testGroup">
+        <xs:sequence />
+      </xs:group>
+    )
+    val pf: ProcessorFactory = Compiler().compileNode(testSchema)
+    assertEquals(s"This basic Schema: $testSchema should compile; here are some diagnostics: ${pf.getDiagnostics}", false, pf.isError)
+    val walker: BasicWalker = new BasicWalker()
+    walker.walkFromRoot(pf.rootView)
+    val nodeStack: List[Either[TermView, TypeView]] = walker.nodeArr.toList
+    assertEquals(s"Node Stack $nodeStack did not have the expected number of elements", 7, nodeStack.size)
+    nodeStack.head match {
+      case Left(personElement: RootView) =>
+        assertEquals("The root element should be named 'PersonData'", "PersonData", personElement.name)
+      case Left(someClass) => fail(s"The Root element was not of type RootView, it was a ${someClass.getClass}!")
+      case Right(someClass) => fail(s"The Root element was not of type RootView, it was a ${someClass.getClass}!")
+    }
+    nodeStack(1) match {
+      case Right(_: ComplexTypeView) =>
+      case Left(someClass) => fail(s"The Root element did not contain a complexType wrapper child, it was a ${someClass.getClass}!")
+      case Right(someClass) => fail(s"The Root element element did not contain a complexType wrapper child, it was a ${someClass.getClass}!")
+    }
+    nodeStack(2) match {
+      case Left(_: SequenceView) =>
+      case Left(someClass) => fail(s"The complexType element did not contain a Sequence child, it was a ${someClass.getClass}!")
+      case Right(someClass) => fail(s"The complexType element did not contain a Sequence child, it was a ${someClass.getClass}!")
+    }
+    nodeStack(3) match {
+      case Left(_: ChoiceView) =>
+      case Left(someClass) => fail(s"The Sequence element did not contain a Choice child, it was a ${someClass.getClass}!")
+      case Right(someClass) => fail(s"The Sequence element did not contain a Choice child, it was a ${someClass.getClass}!")
+    }
+    nodeStack(4) match {
+      case Left(nameElement: ElementBaseView) =>
+        assertEquals("The second child element should be named 'age'", "age", nameElement.name)
+        assertEquals("The second child element should be a simple type", true, nameElement.isSimpleType)
+      case Left(someClass) => fail(s"The Sequence element did not contain a second Element child, it was a ${someClass.getClass}!")
+      case Right(someClass) => fail(s"The Sequence element did not contain a second Element child, it was a ${someClass.getClass}!")
+    }
+    nodeStack(5) match {
+      case Right(simpleType: SimpleTypeView) =>
+        simpleType.primType match {
+          case _:IntView =>
+          case _ => fail(s"The 'age' element should be of simple type Int")
+        }
+      case Left(someClass) => fail(s"The 'age' element did not have a SimpleTypeView, it was a ${someClass.getClass}!")
+      case Right(someClass) => fail(s"The 'age' element did not have a SimpleTypeView, it was a ${someClass.getClass}!")
+    }
+    nodeStack(6) match {
+      case Left(_: GroupRefView) =>
+      case Left(someClass) => fail(s"The Sequence element did not contain a GroupRef child, it was a ${someClass.getClass}!")
+      case Right(someClass) => fail(s"The Sequence element did not contain a GroupRef child, it was a ${someClass.getClass}!")
+    }
+  }

Review comment:
       I agree that it is robust, and I had it this way mainly for the sake of pattern matching and being able to figure out types of specific View elements on the stack when tests were failing.  That being said, for simplicity sake, I think the series of asserts based on `isInstanceOf` and `asInstanceOf` would be best.  I tried implementing a case class as you suggested above, and it turned out to be an even longer solution.




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



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

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455780358



##########
File path: daffodil-core/src/test/scala/org/apache/daffodil/dsom/walker/BasicWalker.scala
##########
@@ -0,0 +1,50 @@
+/*
+ * 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 scala.collection.mutable.ArrayBuffer
+
+class BasicWalker(ignoreTypeWrappers: Boolean = false, onlyElements: Boolean = false) extends AbstractDSOMWalker {
+
+  var nodeArr: ArrayBuffer[Either[TermView, TypeView]] = ArrayBuffer()
+
+  override protected def onWalkBegin(root: RootView): Unit = {}
+
+  override protected def onWalkEnd(root: RootView): Unit = {}
+
+  override def onTermBegin(termElement: TermView): Unit = {
+    if (onlyElements) {
+      termElement match {
+        case _: ElementBaseView => nodeArr += Left(termElement)
+        case _ =>
+      }
+    } else {
+      nodeArr += Left(termElement)
+    }
+  }
+
+  override def onTermEnd(termElement: TermView): Unit = {}
+
+  override def onTypeBegin(typeElement: TypeView): Unit = {
+    if (!ignoreTypeWrappers) {
+      nodeArr += Right(typeElement)
+    }
+  }
+
+  override def onTypeEnd(typeElement: TypeView): Unit = {}
+}

Review comment:
       I'm a little concerned that this ignores a lot of the walker functions. It ignores all End events, and depending on the parameters even more.




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



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

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r456014828



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,174 @@
+/*
+ * 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
+
+/**
+ * Below is a series of "View" Mixins defined such that particular simple attributes
+ * of the core DSOM classes are exposed as a Walk is performed.
+ *
+ * Several of them have a walkDSOM method.  By default, this method is called as the walk is performed,
+ * but a user implementing AbstractDSOMWalker can override this behavior by overriding walkFromRoot
+ * and implementing their own event handlers for each of the DSOM types.
+ */
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+}
+
+trait TypeView
+trait SimpleTypeView extends TypeView {
+  def primType: PrimTypeView
+}
+trait ComplexTypeView extends TypeView {
+  def group: ModelGroupView
+}
+
+trait ModelGroupView extends TermView {
+  def groupMembers: Seq[TermView]
+}
+
+trait ChoiceView extends ModelGroupView
+trait SequenceView extends ModelGroupView
+trait GroupRefView extends ModelGroupView {
+  def isHidden: Boolean

Review comment:
       Yeah, I agree yoru concern, that does feel like things could get more confusing, and DSOM is alrady conufsing enough as it is.
   
   An another approach with maybe less confusion would be to create case classes for all our Views. Accessing a View would result in allocating a new case class and copying relevant information from the base class into the View class, which would allow modifcations/filtering to what gets copied. Though, this could result in lots of allocations, which is probably best to avoided.
   
   I'd say these types of changes probably aren't worth it unless we find more examples where it would be useful to have the View be substantially different. Probalby not worth it for just this one case.




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455968343



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,177 @@
+/*
+ * 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
+
+/**
+ * Below is a series of "View" Mixins defined such that particular simple attributes
+ * of the core DSOM classes are exposed as a Walk is performed.
+ *
+ * Several of them have a walkDSOM method.  By default, this method is called as the walk is performed,
+ * but a user implementing AbstractDSOMWalker can override this behavior by overriding walkFromRoot
+ * and implementing their own event handlers for each of the DSOM types.
+ */
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+}
+
+trait TypeView
+trait SimpleTypeView extends TypeView {
+  def primType: PrimTypeView
+}
+trait ComplexTypeView extends TypeView {
+  def group: ModelGroupView
+}
+
+trait ModelGroupView extends TermView {
+  def groupMembers: Seq[TermView]
+}
+
+trait ChoiceView extends ModelGroupView
+trait SequenceView extends ModelGroupView
+trait GroupRefView extends ModelGroupView {
+  def isHidden: Boolean
+}
+
+trait ElementDeclView extends CommonContextView {
+  def isSimpleType: Boolean
+  def isComplexType: Boolean
+  def simpleType: SimpleTypeView
+  def complexType: ComplexTypeView
+  def hasDefaultValue: Boolean
+}
+
+trait ElementBaseView extends ElementDeclView with TermView {
+  def name: String
+}
+
+trait RootView extends ElementBaseView
+
+/**
+ * A class designed to walk the internal representation of a DFDL Schema File.
+ *
+ * There are 2 main event handlers an implementing class has to worry about: one for Terms, and
+ * another for Types.  These are called as the DSOM is walked.
+ *
+ * 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.
+ */
+abstract class AbstractDSOMWalker {
+
+  /**
+   * 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: RootView): 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
+   */
+  protected def onWalkEnd(root: RootView): Unit
+
+  /**
+   * Method to be called whenever any element that is a Term is encountered.
+   * This applies to Sequence, Choice, GroupRef, ElementBase, etc.
+   *
+   * It is highly recommended that, when implementing this method, you pattern match
+   * some of these different sub-types at some point to handle each accordingly.
+   *
+   * @param termElement the term element
+   */
+  def onTermBegin(termElement: TermView): Unit
+
+  /**
+   * Method to be called when a Term element has finished processing
+   *
+   * @param termElement the term element
+   */
+  def onTermEnd(termElement: TermView): Unit
+
+  /**
+   * Method to be called when either a Simple or Complex Type element has been encountered.
+   * This is just the element for <simpleType> or <complexType>, so no implementation is necessary if you
+   * do not care about these wrapper portion; its children will be walked automatically.
+   *
+   * @param typeElement either a Simple or Complex type
+   */
+  def onTypeBegin(typeElement: TypeView): Unit
+
+  /**
+   * Method to be called when either a Simple or Complex Type element has been finished
+   * processing.  See onTypeBegin method description
+   *
+   * @param typeElement either a Simple or Complex type
+   */
+  def onTypeEnd(typeElement: TypeView): Unit
+
+  /**
+   * Starts a DSOM walk from the Root object.  By default, the walk is completed recursively in the helper
+   * method below, but a custom walk can be defined by overriding
+   * this method instead and calling local event handlers.
+   * @param schemaSetRoot The root element of the DFDL Schema.  This will be the starting point of the traversal
+   */
+  def walkFromRoot(schemaSetRoot: RootView): Unit = {
+    onWalkBegin(schemaSetRoot)
+    walkerHelper(schemaSetRoot)
+    // onWalkEnd will return the value returned by this method!

Review comment:
       Agreed - Resolved in the latest revision




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r456009313



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,174 @@
+/*
+ * 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
+
+/**
+ * Below is a series of "View" Mixins defined such that particular simple attributes
+ * of the core DSOM classes are exposed as a Walk is performed.
+ *
+ * Several of them have a walkDSOM method.  By default, this method is called as the walk is performed,
+ * but a user implementing AbstractDSOMWalker can override this behavior by overriding walkFromRoot
+ * and implementing their own event handlers for each of the DSOM types.
+ */
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+}
+
+trait TypeView
+trait SimpleTypeView extends TypeView {
+  def primType: PrimTypeView
+}
+trait ComplexTypeView extends TypeView {
+  def group: ModelGroupView
+}
+
+trait ModelGroupView extends TermView {
+  def groupMembers: Seq[TermView]
+}
+
+trait ChoiceView extends ModelGroupView
+trait SequenceView extends ModelGroupView
+trait GroupRefView extends ModelGroupView {
+  def isHidden: Boolean

Review comment:
       I think there is a solution to this, but it would break the current paradigm of every field in the `View`s referencing an existing field.  We could replace the existing `group` element in `ComplexTypeView` with something like `optVisibleGroup`, and then the implementation for `optVisibleGroup` would pattern-match `group`; if it is hidden, return `None, otherwise return the `group` wrapped in an option.  Similarly, instead of `groupMembers` being in `TermView`, we would have `visisbleGroupMembers`, and in the implementation `visibleGroupMembers` would be a filtered list of the real `groupMembers`, excluding any `TermView` that is a `GroupRef` and is also hidden.
   
   However, my main concern with this is it would potentially cause more confusion, leading the end user to wonder why we're only allowed to see "visible" members, and might make traversal very annoying with having to constantly unpack Options.




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453130366



##########
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:
       I agree, I definitely want to separate all of the NIFI-related code from anything related to DSOM.  I am working on NIFI Processors that use this entire DSOM API but they (more so the Unparse processor) are not quite finished yet, I am still doing a great deal of refactoring.  
   
   I am not sure that any of the NIFI code either than RecordWalker and RecordSchemaNode need exist in the main Daffodil codebase - if it were to go here, would it still go in core but just a different subdirectory, or would the japi/sapi modules be more appropriate?  
   
   Since the base of my code largely builds off of the repo you linked, let me know if you think it is appropriate if I simply rebase my project onto that one or if we should go with a different set of processors since they are a good bit different when they utilize NIFI Records internally.




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



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

Posted by GitBox <gi...@apache.org>.
tuxji commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r452932910



##########
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:
       Yes, I was trying to reinforce what Steve said about the AbstractWalker becoming a handler of events, not doing any actual walking itself.  There are pros and cons to both visitors/walkers and listeners as people have blogged about for ANTLR, but I found the listener was easier to use for the ANTLR grammar and its parse tree I was working with.




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



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

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455886813



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,177 @@
+/*
+ * 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
+
+/**
+ * Below is a series of "View" Mixins defined such that particular simple attributes
+ * of the core DSOM classes are exposed as a Walk is performed.
+ *
+ * Several of them have a walkDSOM method.  By default, this method is called as the walk is performed,
+ * but a user implementing AbstractDSOMWalker can override this behavior by overriding walkFromRoot
+ * and implementing their own event handlers for each of the DSOM types.
+ */
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+}
+
+trait TypeView
+trait SimpleTypeView extends TypeView {
+  def primType: PrimTypeView
+}
+trait ComplexTypeView extends TypeView {
+  def group: ModelGroupView
+}
+
+trait ModelGroupView extends TermView {
+  def groupMembers: Seq[TermView]
+}
+
+trait ChoiceView extends ModelGroupView
+trait SequenceView extends ModelGroupView
+trait GroupRefView extends ModelGroupView {
+  def isHidden: Boolean
+}
+
+trait ElementDeclView extends CommonContextView {
+  def isSimpleType: Boolean
+  def isComplexType: Boolean
+  def simpleType: SimpleTypeView
+  def complexType: ComplexTypeView
+  def hasDefaultValue: Boolean

Review comment:
       If NiFi doesn't need it, I'd vote for just removing it. We can always add it in the future if some other walker implementation needs it.




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



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

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455763049



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,177 @@
+/*
+ * 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
+
+/**
+ * Below is a series of "View" Mixins defined such that particular simple attributes
+ * of the core DSOM classes are exposed as a Walk is performed.
+ *
+ * Several of them have a walkDSOM method.  By default, this method is called as the walk is performed,
+ * but a user implementing AbstractDSOMWalker can override this behavior by overriding walkFromRoot
+ * and implementing their own event handlers for each of the DSOM types.
+ */
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+}
+
+trait TypeView
+trait SimpleTypeView extends TypeView {
+  def primType: PrimTypeView
+}
+trait ComplexTypeView extends TypeView {
+  def group: ModelGroupView
+}
+
+trait ModelGroupView extends TermView {
+  def groupMembers: Seq[TermView]
+}
+
+trait ChoiceView extends ModelGroupView
+trait SequenceView extends ModelGroupView
+trait GroupRefView extends ModelGroupView {
+  def isHidden: Boolean
+}
+
+trait ElementDeclView extends CommonContextView {
+  def isSimpleType: Boolean
+  def isComplexType: Boolean
+  def simpleType: SimpleTypeView
+  def complexType: ComplexTypeView
+  def hasDefaultValue: Boolean
+}
+
+trait ElementBaseView extends ElementDeclView with TermView {
+  def name: String
+}
+
+trait RootView extends ElementBaseView
+
+/**
+ * A class designed to walk the internal representation of a DFDL Schema File.
+ *
+ * There are 2 main event handlers an implementing class has to worry about: one for Terms, and
+ * another for Types.  These are called as the DSOM is walked.
+ *
+ * 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.
+ */
+abstract class AbstractDSOMWalker {
+
+  /**
+   * 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: RootView): 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
+   */
+  protected def onWalkEnd(root: RootView): Unit
+
+  /**
+   * Method to be called whenever any element that is a Term is encountered.
+   * This applies to Sequence, Choice, GroupRef, ElementBase, etc.
+   *
+   * It is highly recommended that, when implementing this method, you pattern match
+   * some of these different sub-types at some point to handle each accordingly.
+   *
+   * @param termElement the term element
+   */
+  def onTermBegin(termElement: TermView): Unit
+
+  /**
+   * Method to be called when a Term element has finished processing
+   *
+   * @param termElement the term element
+   */
+  def onTermEnd(termElement: TermView): Unit
+
+  /**
+   * Method to be called when either a Simple or Complex Type element has been encountered.
+   * This is just the element for <simpleType> or <complexType>, so no implementation is necessary if you
+   * do not care about these wrapper portion; its children will be walked automatically.
+   *
+   * @param typeElement either a Simple or Complex type
+   */
+  def onTypeBegin(typeElement: TypeView): Unit
+
+  /**
+   * Method to be called when either a Simple or Complex Type element has been finished
+   * processing.  See onTypeBegin method description
+   *
+   * @param typeElement either a Simple or Complex type
+   */
+  def onTypeEnd(typeElement: TypeView): Unit
+
+  /**
+   * Starts a DSOM walk from the Root object.  By default, the walk is completed recursively in the helper
+   * method below, but a custom walk can be defined by overriding
+   * this method instead and calling local event handlers.
+   * @param schemaSetRoot The root element of the DFDL Schema.  This will be the starting point of the traversal
+   */
+  def walkFromRoot(schemaSetRoot: RootView): Unit = {
+    onWalkBegin(schemaSetRoot)
+    walkerHelper(schemaSetRoot)
+    // onWalkEnd will return the value returned by this method!

Review comment:
       Can remove this comment now that walkerEnd doesn't return anything.




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



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

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r456002153



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,174 @@
+/*
+ * 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
+
+/**
+ * Below is a series of "View" Mixins defined such that particular simple attributes
+ * of the core DSOM classes are exposed as a Walk is performed.
+ *
+ * Several of them have a walkDSOM method.  By default, this method is called as the walk is performed,
+ * but a user implementing AbstractDSOMWalker can override this behavior by overriding walkFromRoot
+ * and implementing their own event handlers for each of the DSOM types.
+ */
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+}
+
+trait TypeView
+trait SimpleTypeView extends TypeView {
+  def primType: PrimTypeView
+}
+trait ComplexTypeView extends TypeView {
+  def group: ModelGroupView
+}
+
+trait ModelGroupView extends TermView {
+  def groupMembers: Seq[TermView]
+}
+
+trait ChoiceView extends ModelGroupView
+trait SequenceView extends ModelGroupView
+trait GroupRefView extends ModelGroupView {
+  def isHidden: Boolean

Review comment:
       One last look over this (looking really good, btw), I'm wondering if we want isHidden visible? We could in theory have daffodil-core present a view of groupMembers where hidden group refs are filtered out. And then API users never even have to be aware that some things are hidden because they'll never see them. Right now that logic is handled by the AbstractDSOMWalker, but if a user didn't use that they'd have to keep hidden group refs in consideration.
   
   I think the main issue with this is that daffodil-core can't actually modify the underlying groupMembers. Daffodil still DSOM still needs to know about hidden groups, and there is no way to actually present a View different from the concrete class. The View can only *hide* members--it can't actually make changes to those members.
   
   I guess we just don't have an issue anywhere else where a View member needs a different type/content than the underlying member.
   
   Not sure of a good way to handle this unless we start introduce different names for View members or allocating new classes, which probably starts getting messy. So maybe this isn't worth considering, and just something to keep in mind with how this View API works.




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455970514



##########
File path: daffodil-core/src/test/scala/org/apache/daffodil/dsom/walker/BasicWalker.scala
##########
@@ -0,0 +1,50 @@
+/*
+ * 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 scala.collection.mutable.ArrayBuffer
+
+class BasicWalker(ignoreTypeWrappers: Boolean = false, onlyElements: Boolean = false) extends AbstractDSOMWalker {
+
+  var nodeArr: ArrayBuffer[Either[TermView, TypeView]] = ArrayBuffer()
+
+  override protected def onWalkBegin(root: RootView): Unit = {}
+
+  override protected def onWalkEnd(root: RootView): Unit = {}
+
+  override def onTermBegin(termElement: TermView): Unit = {
+    if (onlyElements) {
+      termElement match {
+        case _: ElementBaseView => nodeArr += Left(termElement)
+        case _ =>
+      }
+    } else {
+      nodeArr += Left(termElement)
+    }
+  }
+
+  override def onTermEnd(termElement: TermView): Unit = {}
+
+  override def onTypeBegin(typeElement: TypeView): Unit = {
+    if (!ignoreTypeWrappers) {
+      nodeArr += Right(typeElement)
+    }
+  }
+
+  override def onTypeEnd(typeElement: TypeView): Unit = {}
+}

Review comment:
       That makes sense, I did overlook that.  In the latest revision I wound up simply adding the element both when the begin and end events are invoked, and then the refactored test for complex types verifies the elements corresponding to those end events occur in the expected indices in the list.  I wound up just merging these extra assertions with the existing test for complex types - let me know if you think there should be an explicit separate test focused on pairs of start and end events.




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



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

Posted by GitBox <gi...@apache.org>.
mbeckerle commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453010086



##########
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:
       There needs to be a way to "register" a walker. I.e., a Daffodil API call where the walker is passed to Daffodil, so that it can be invoked. Or if the control pattern is that the walker takes control, it's really just a hook where daffodil informs the walker that the DSOM tree is ready to be walked. This should allow for many external walkers to be registered, not just one. They should be called in turn. 
   
   Then all the NiFi-specific logic, tests, etc. all goes into distinct separate code that is not part of Daffodil. It becomes an application of Daffodil's libraries within NiFi code. 




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453130366



##########
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:
       I agree, I definitely want to separate all of the NIFI-related code from anything related to DSOM.  I am working on NIFI Processors that use this entire DSOM API but they (more so the Unparse processor) are not quite finished yet, I am still doing a great deal of refactoring.  
   
   I am not sure that any of the NIFI code other than RecordWalker and RecordSchemaNode need exist in the main Daffodil codebase - if it were to go here, would it still go in core but just a different subdirectory, or would the japi/sapi modules be more appropriate?  
   
   Since the base of my code largely builds off of the repo you linked, let me know if you think it is appropriate if I simply rebase my project onto that one or if we should go with a different set of processors since they are a good bit different when they utilize NIFI Records internally.




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453840116



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/Term.scala
##########
@@ -18,16 +18,19 @@
 package org.apache.daffodil.dsom
 
 import java.util.UUID
+
 import org.apache.daffodil.exceptions.Assert
 import org.apache.daffodil.grammar.TermGrammarMixin
 import org.apache.daffodil.schema.annotation.props.gen.YesNo
-import java.lang.{ Integer => JInt }
+import java.lang.{Integer => JInt}

Review comment:
       Sorry, I'm not sure why IntelliJ changed this!  I'll change it back in the next commit - do you all have a styling XML file that I can use for ScalaStyle or a similar tool?




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453876865



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/SimpleTypes.scala
##########
@@ -53,9 +54,9 @@ trait TypeBase {
 trait NonPrimTypeMixin
 
 sealed trait SimpleTypeBase extends TypeBase
-  with HasOptRepTypeMixin {
+  with HasOptRepTypeMixin with SimpleTypeView {
 
-  def primType: PrimType
+  override def primType: PrimType

Review comment:
       I can understand how that could cause some issues.  What would be a more stable way to access information about a simple type without exposing the internal classes?  Would it be enough to simply add a String member of SimpleTypeBase that returns `primType.toString`?  My NiFi Record implementation is very dependent on pattern matching Simple types, and I imagine most implementations would be as well.




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



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

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453890061



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,163 @@
+/*
+ * 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 org.apache.daffodil.dpath.NodeInfo.PrimType
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+  def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {

Review comment:
       Actually, these walkDSOM functions can't really be easily modified since the objects that implement the *View traits are provided by Daffodil. If we want the user to be able to implement custom walking logic, we probably need to go back something like how it was before where the Walker has the walk logic rather than the Views.




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455459024



##########
File path: project/Dependencies.scala
##########
@@ -29,6 +29,7 @@ object Dependencies {
     "xml-resolver" % "xml-resolver" % "1.2",
     "commons-io" % "commons-io" % "2.6",
     "jline" % "jline" % "2.14.6",
+    "org.scalatest" %% "scalatest" % "3.2.0" % "test"

Review comment:
       Noted, I did not realize the flaws of ScalaTest; the particular test cases I wrote happened to be helpful for representing the structure of the large schema files I wanted to verify I had the correct NiFi RecordSchema for, but I realize now how it might be more alienating for non-native English speakers, plus Unit Tests shouldn't be based around very large structures happening to work and should be more focused.  
   
   That all said, I developed a new testing file in the latest revision that uses JUnits and while it is much simpler than what I had before. it is still a bit robust in that it checks to see if all the important Simple and Complex View types work as expected.  It is also based around a much simpler implementation of the Abstract Walker.  Feel free to comment on it.




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455970514



##########
File path: daffodil-core/src/test/scala/org/apache/daffodil/dsom/walker/BasicWalker.scala
##########
@@ -0,0 +1,50 @@
+/*
+ * 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 scala.collection.mutable.ArrayBuffer
+
+class BasicWalker(ignoreTypeWrappers: Boolean = false, onlyElements: Boolean = false) extends AbstractDSOMWalker {
+
+  var nodeArr: ArrayBuffer[Either[TermView, TypeView]] = ArrayBuffer()
+
+  override protected def onWalkBegin(root: RootView): Unit = {}
+
+  override protected def onWalkEnd(root: RootView): Unit = {}
+
+  override def onTermBegin(termElement: TermView): Unit = {
+    if (onlyElements) {
+      termElement match {
+        case _: ElementBaseView => nodeArr += Left(termElement)
+        case _ =>
+      }
+    } else {
+      nodeArr += Left(termElement)
+    }
+  }
+
+  override def onTermEnd(termElement: TermView): Unit = {}
+
+  override def onTypeBegin(typeElement: TypeView): Unit = {
+    if (!ignoreTypeWrappers) {
+      nodeArr += Right(typeElement)
+    }
+  }
+
+  override def onTypeEnd(typeElement: TypeView): Unit = {}
+}

Review comment:
       That makes sense, I did overlook that.  I wound up simply adding the element both when the begin and end events are invoked, and then the refactored test for complex types verifies the elements corresponding to those end events occur in the expected indices in the list.




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



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

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455882154



##########
File path: daffodil-core/src/test/scala/org/apache/daffodil/dsom/walker/BasicWalker.scala
##########
@@ -0,0 +1,50 @@
+/*
+ * 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 scala.collection.mutable.ArrayBuffer
+
+class BasicWalker(ignoreTypeWrappers: Boolean = false, onlyElements: Boolean = false) extends AbstractDSOMWalker {
+
+  var nodeArr: ArrayBuffer[Either[TermView, TypeView]] = ArrayBuffer()
+
+  override protected def onWalkBegin(root: RootView): Unit = {}
+
+  override protected def onWalkEnd(root: RootView): Unit = {}
+
+  override def onTermBegin(termElement: TermView): Unit = {
+    if (onlyElements) {
+      termElement match {
+        case _: ElementBaseView => nodeArr += Left(termElement)
+        case _ =>
+      }
+    } else {
+      nodeArr += Left(termElement)
+    }
+  }
+
+  override def onTermEnd(termElement: TermView): Unit = {}
+
+  override def onTypeBegin(typeElement: TypeView): Unit = {
+    if (!ignoreTypeWrappers) {
+      nodeArr += Right(typeElement)
+    }
+  }
+
+  override def onTypeEnd(typeElement: TypeView): Unit = {}
+}

Review comment:
       Understood, but we should still have tests that verify that end events are happening correctly. If there were a bug in the AbstractDSOMWalker that caused certain end events to never happen or to happen out of order, we wouldn't know about it. Whether this test walker does those checks or if it's handled by in each test itself doesn't really seem that important, but something should be able to verify that it's correct.




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453128358



##########
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:
       While I did leave the fundamental recursive walk structure intact, as Mike stated might have to do anyway in the end, I did consolidate quite a few methods.  Now the user need only implement events for starting and ending the walk, and 2 handlers each for Terms and Types, where a Type could either be Simple or Complex.  See the latest commit.
   
   This was a very good suggestion, as just doing that makes the code a lot cleaner!  Let me know if you think I implemented it correctly.




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



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

Posted by GitBox <gi...@apache.org>.
mbeckerle commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455098824



##########
File path: project/Dependencies.scala
##########
@@ -29,6 +29,7 @@ object Dependencies {
     "xml-resolver" % "xml-resolver" % "1.2",
     "commons-io" % "commons-io" % "2.6",
     "jline" % "jline" % "2.14.6",
+    "org.scalatest" %% "scalatest" % "3.2.0" % "test"

Review comment:
       I would go farther than this. I really hate scalatest. It is a pointless DSL that enables English-centric people to use English-centric sentence structure words like "should" which is unnatural in other languages (words like should become verb conjugation in some languages), to express tests. This is unfriendly to non-native English speakers. 
   
   I think scalatest primarily serves as an example of how to program DSLs in scala. As a test rig itself it's unnecessary, and in fact problematic. 
   
   It encourages tests to contain text strings that are for reporting purposes, but those strings must be kept consistent with the test itself. 
   It is better to just choose a good name for the test method and test class. 
   
   This is from the scalatest web page. I've added my comments:
   
   ```
   class ExampleSpec extends AnyFlatSpec with should.Matchers {
   
     "A Stack" should "pop values in last-in-first-out order" in { 
       //
       // This English-centric phrasing has no advantage over @Test def stackShouldPopInLIFOOrder()
       //
       val stack = new Stack[Int]
       stack.push(1)
       stack.push(2)
       stack.pop() should be (2)
       stack.pop() should be (1)
     }
   
     it should "throw NoSuchElementException if an empty stack is popped" in { 
       //
       // it? Referring to what? "A Stack" is conveniently right there above in this toy example. 
       // In real code it could be off screen scrolled away. This is not a strength of natural language we 
       // want to pull into formal language. 
       // 
       // The above also repeats the exception name which is 100% redundant and as that string is just for a report,
       // it can become inconsistent with the way the test evolves if the exception thrown is changed.
   
       val emptyStack = new Stack[Int]
       a [NoSuchElementException] should be thrownBy {
         emptyStack.pop()
       } 
   
      //
      // This has no advantage over @Test def stackThrowsProperException() ....
      //
     }
   }
   
   ```
   
   
   
   
   I think scalatest primarily serves as an example of how to program DSLs in scala. As a test rig itself it's unnecessary, and in fact problematic. 




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453982477



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/SimpleTypes.scala
##########
@@ -53,9 +54,9 @@ trait TypeBase {
 trait NonPrimTypeMixin
 
 sealed trait SimpleTypeBase extends TypeBase
-  with HasOptRepTypeMixin {
+  with HasOptRepTypeMixin with SimpleTypeView {
 
-  def primType: PrimType
+  override def primType: PrimType

Review comment:
       I went ahead and defined an Enum type called `PrimTypeView` and created Enum representations for all the "PrimType" enum types within it.  I could not find a way to make the NodeInfo class import *anything* from `dsom.walker`, so I was unable to make it extend any trait I defined there.  Instead, I added a new field to `SimpleTypeBase` called `primTypeView` which just matches `primType` to my new Enum type.  Let me know if you think this approach should be changed; the pattern matching is done in `SimpleTypeBase` itself.




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453838690



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,163 @@
+/*
+ * 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 org.apache.daffodil.dpath.NodeInfo.PrimType
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+  def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {

Review comment:
       The walk is currently initiated here:
   `final def walkFromRoot(schemaSetRoot: RootView): T = {
       onWalkBegin(schemaSetRoot)
       schemaSetRoot.walkDSOM(this)
       // onWalkEnd will return the value returned by this method!
       onWalkEnd(schemaSetRoot)
     }`
   As you can see, this method requires `walkDSOM` to be a public member of `TermView`.  I am not quite sure how exactly the walk would begin without this method being public.




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453865556



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,163 @@
+/*
+ * 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 org.apache.daffodil.dpath.NodeInfo.PrimType
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+  def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    walker.onTermEnd(this)
+  }
+}
+
+trait SimpleTypeView {
+  def primType: PrimType
+}
+trait ComplexTypeView {
+  def group: ModelGroupView
+}
+
+trait ModelGroupView extends TermView {
+  def groupMembers: Seq[TermView]
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    this.groupMembers.foreach(_.walkDSOM(walker))
+    walker.onTermEnd(this)
+  }
+}
+
+trait ChoiceView extends ModelGroupView
+trait SequenceView extends ModelGroupView
+trait GroupRefView extends ModelGroupView {
+  def isHidden: Boolean
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = if (!isHidden) super.walkDSOM(walker)
+}
+
+trait ElementDeclView extends CommonContextView {
+  def isSimpleType: Boolean
+  def isComplexType: Boolean
+  def simpleType: SimpleTypeView
+  def complexType: ComplexTypeView
+  def hasDefaultValue: Boolean
+}
+
+trait ElementBaseView extends ElementDeclView with TermView {
+  def name: String
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    if (isComplexType) {
+      walker.onTypeBegin(Right(complexType))
+      complexType.group.walkDSOM(walker)
+      walker.onTypeBegin(Right(complexType))
+    } else {
+      walker.onTypeBegin(Left(simpleType))
+      walker.onTypeBegin(Left(simpleType))
+    }
+    walker.onTermEnd(this)
+  }
+
+}
+
+trait RootView extends ElementBaseView
+
+/**
+ * A class designed to walk the internal representation of a DFDL Schema File.
+ *
+ * There are 2 main event handlers an implementing class has to worry about: one for Terms, and
+ * another for Types.  These are called as the DSOM is walked.
+ *
+ * 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.
+ * @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] {
+
+  /**

Review comment:
       It is possible to build nars using scala code, I previously had all of this code in a maven project that mixed NIFI-specific Java code and Scala code.  It only required a few additional dependencies to include.
   
   That being said, if the NiFi developer doesn't have to code in Scala, they don't have to worry about any of that - they would only need to do this configuration if they wanted to implement AbstractDSOMWalker in Scala




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



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

Posted by GitBox <gi...@apache.org>.
mbeckerle commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453006731



##########
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:
       Sometimes nothing but a recursive walk algorithm will do. A visitor struggles with state management when it has to refer to more than one tree node at a time. 
   
   Suggest we stick for now with what makes this particular NiFI Record-Schema synthesizer work out well. If that's a visit pattern then great, but if an actual walk proves more natural we should have that. 




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455885398



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,177 @@
+/*
+ * 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
+
+/**
+ * Below is a series of "View" Mixins defined such that particular simple attributes
+ * of the core DSOM classes are exposed as a Walk is performed.
+ *
+ * Several of them have a walkDSOM method.  By default, this method is called as the walk is performed,
+ * but a user implementing AbstractDSOMWalker can override this behavior by overriding walkFromRoot
+ * and implementing their own event handlers for each of the DSOM types.
+ */
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+}
+
+trait TypeView
+trait SimpleTypeView extends TypeView {
+  def primType: PrimTypeView
+}
+trait ComplexTypeView extends TypeView {
+  def group: ModelGroupView
+}
+
+trait ModelGroupView extends TermView {
+  def groupMembers: Seq[TermView]
+}
+
+trait ChoiceView extends ModelGroupView
+trait SequenceView extends ModelGroupView
+trait GroupRefView extends ModelGroupView {
+  def isHidden: Boolean
+}
+
+trait ElementDeclView extends CommonContextView {
+  def isSimpleType: Boolean
+  def isComplexType: Boolean
+  def simpleType: SimpleTypeView
+  def complexType: ComplexTypeView
+  def hasDefaultValue: Boolean

Review comment:
       My current NiFi Walker doesn't need this; I merely included it because I was trying to mimic the structure of `ElementDeclMixin`.  I notice that `ElementDeclMixin` has `defaultValueAsString` member, would that be the correct member to add?  If so, then I could add it, but I could also just remove `hasDefaultValue` from the ElementDeclView if this is not an important attribute.




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453986218



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,163 @@
+/*
+ * 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 org.apache.daffodil.dpath.NodeInfo.PrimType
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+  def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {

Review comment:
       I could always change `walkFromRoot` to a non-final method.  This would allow the user to override it and then instead of doing `schemaSetRoot.walkDSOM(this);` they could call their own custom helper method, similar to what we had before.  Keeping the current code intact while doing this would enable the existing walking strategy to be the default while also allowing the user to override with their own implementation. 




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



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

Posted by GitBox <gi...@apache.org>.
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



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

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455040538



##########
File path: daffodil-core/src/test/scala/org/apache/daffodil/dsom/walker/TestWalker.scala
##########
@@ -0,0 +1,335 @@
+/*
+ * 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 scala.collection.mutable.ListBuffer
+
+/**
+ * This represents all possible types a Field could be; the View Mixin is pattern matched to one
+ * of these in TestNode
+ */
+object FieldType extends Enumeration {
+  type FieldType = Value
+  val RecordType, ChoiceType, ArrayType, DateTimeType, DateType, TimeType, BooleanType, ByteType,
+  ShortType, IntType, LongType, BigIntType, FloatType, DoubleType, StringType = Value
+}
+
+import org.apache.daffodil.dsom.walker.FieldType._
+
+/**
+ * Represents the DataType of a TestField; note that all of the Simple types manifest as
+ * a SimpleDataType with a single FieldType containing the FieldType enum for the simple.
+ * @param fieldType the particular FieldType Enum that this DataType backs.
+ */
+class DataType(val fieldType: FieldType.Value)
+case class RecordDataType(var childSchema: TestSchema) extends DataType(RecordType)
+case class ChoiceDataType(var possibleSubTypes: List[DataType]) extends DataType(ChoiceType)
+case class ArrayDataType(var elementType: DataType) extends DataType(ArrayType)
+case class SimpleDataType(simpleType: FieldType) extends DataType(simpleType)
+
+/**
+ * A field within a TestSchema
+ * @param fieldName name of the field
+ * @param dataType dataType of the field
+ */
+class TestField(val fieldName: String, val dataType: DataType)
+
+/**
+ * Base Schema object for this test class.  The only part that is really used is the field list;
+ * there are some helper methods based around the list
+ * @param name the name for this TestSchema
+ * @param namespace the target namespace of this TestSchema
+ * @param fields the list of fields that define this TestSchema
+ */
+class TestSchema(val name: String, val namespace: String, val fields: List[TestField]) {
+  def getField(name: String): Option[TestField] =
+    fields.filter(field => field.fieldName == name) match {
+      case List(result) => Some(result)
+      case _ => None
+    }
+  val fieldCount: Int = fields.size
+  val fieldNames: List[String] = fields.map(field => field.fieldName)
+}
+
+
+/**
+ * @param fieldName the name for the RecordField
+ * @param dataType the DataType of this RecordField
+ */
+class OptionalTestField(fieldName: String, dataType: DataType)
+  extends TestField(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: TestField) = this(recordField.fieldName, recordField.dataType)
+}
+
+/**
+ * Concrete implementation of the AbstractDSOMWalker abstract class.
+ * This class produces a TestSchema that is intended to match the original DFDL file.
+ *
+ * The TestSchema is built in 3 primary stages:
+ * 1) A tree of TestNodes 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 TestNodes undergoes some post-processing, mainly to remove redundant Record wrappers.
+ * 3) The tree of TestNodes is converted into a TestSchema; it is walked recursively within this class.
+ */
+class TestWalker extends AbstractDSOMWalker {
+
+  // this is the critical data structure for managing the temporary TestNodes that are created
+  // when the Schema is initially walked.  This will then be converted to the actual TestSchema.
+  private var objectStack: List[TestNode] = List()
+
+  // this will contain the final TestSchema after step 3 as described above is complete.
+  private var result: Option[TestSchema] = None
+  def getResult: TestSchema = result.orNull
+  def stringRep: String = if (result.isDefined) result.get.toString else ""
+
+  /**
+   * Every event handler for onXBegin is routed to this helper method.  Individual properties of
+   * a given element are not actually handled here; these properties are extracted when a
+   * TestNode is instantiated
+   * @param dsomElement the particular element that was encountered.  Could be essentially any Term element.
+   */
+  private def onNodeBegin(dsomElement: TermView): Unit = {
+    val newNode: TestNode = new TestNode(dsomElement)
+    // we need add the new node as a new child of whatever is currently at the top of the stack
+    objectStack.head.addChild(newNode)
+    // however, we also add the node to the stack itself!  We need to be able to add children to it
+    // if it is, say, another record or array.
+    objectStack = newNode +: objectStack
+  }
+
+  /**
+   * Every event handler for onXEnd is routed to this helper method.  We don't need to do anything other
+   * than remove the top element of the stack; this should be the current element.
+   */
+  private def onNodeEnd(): Unit = objectStack = objectStack.tail
+
+  override def onTermBegin(termElement: TermView): Unit = termElement match {
+    case _: SequenceView | _: ChoiceView | _: ElementBaseView => onNodeBegin(termElement)
+    case _ =>
+  }
+
+  override def onTermEnd(termElement: TermView): Unit = termElement match {
+    case _: SequenceView | _: ChoiceView | _: ElementBaseView => onNodeEnd()
+    case _ =>
+  }
+
+  override def onTypeBegin(typeElement: TypeView): Unit = {}
+
+  override def onTypeEnd(typeElement: TypeView): Unit = {}
+
+  override def onWalkBegin(root: RootView): Unit = objectStack = List(new TestNode(root))
+
+  override def onWalkEnd(root: RootView): Unit = {
+    // After the walk is over, we perform postProcessing and then convert the TestNode tree
+    // into a TestSchema.  Also, if we are in dev. mode, we print out the TestNode tree
+    if (!RecordWalker.PRODUCTION_MODE) println(objectStack.head)
+    postProcessing()
+    if (!RecordWalker.PRODUCTION_MODE) println(objectStack.head)
+    result = Some(getRecordSchema(objectStack.head))
+  }
+
+  /**
+   * Determines if a Record is "extra"; that is, if it should be replaced with
+   * its list of children SchemaNodes within whatever parent TestNode it's a part of
+   * @param childNode the node to be considered
+   * @return true if this node is a Record and has no name, false otherwise
+   */
+  private def isExtraRecord(parentNode: TestNode, childNode: TestNode): Boolean = {
+    // any no-name nodes with no children are immediately removed
+    (childNode.name.isEmpty && childNode.children.isEmpty) || {
+      parentNode.fieldType match {
+        case RecordType | ArrayType =>
+          childNode.fieldType match {
+            // This removes extra wrapper records around children of records or arrays
+            // usually used to remove things like the DFDL complexType, simpleType elements
+            case RecordType => childNode.name.isEmpty
+            case _ => false
+          }
+        // Currently, all double choices are removed.  This was mainly done to make GroupRefs work
+        // for the JPEG Schema, but may not be the correct approach for all cases.
+        case ChoiceType =>
+          childNode.fieldType match {
+            case ChoiceType => true
+            case _ => false
+          }
+        case _ => false
+      }
+    }
+
+  }
+
+  /**
+   * Recursively replace any SchemaNodes that are of type record and do not have
+   * a name attribute with their children. These usually represent unnecessary wrapper nodes or empty
+   * records with no elements.
+   *
+   * Given a TestNode, we copy over all the children to a list, and initialize another empty list.
+   * Then, for every Node in the first list, if it is considered "extra",
+   * then its *children* are added to the second list.
+   * Otherwise, the Node itself is added.
+   *
+   * Then, we let the first list equal the second and we repeat this process
+   * until none of the children are "extra".
+   * This is necessary because sometimes we can have towers of extra Nodes that would
+   * never get resolved if we just took care of 1 or 2 layers; all must be dealt with at once.
+   *
+   * Then, the current TestNode's children list is replaced with this filtered one,
+   * and this method is called on all the items of that list.
+   * @param schemaNode the current Node undergoing the algorithm described above
+   */
+  private def removeExtraRecords(schemaNode: TestNode): Unit = {
+    schemaNode.fieldType match {
+      // Only SchemaNodes that are these 3 types should be able to have child nodes.
+      case RecordType | ChoiceType | ArrayType =>
+        if (schemaNode.children.exists(child => isExtraRecord(schemaNode, child))) {
+          var currChildren: ListBuffer[TestNode] = new ListBuffer[TestNode]()
+          for (child <- schemaNode.children) currChildren += child
+          while (currChildren.exists(child => isExtraRecord(schemaNode, child))) {
+            val toCopy: ListBuffer[TestNode] = new ListBuffer[TestNode]()
+            for (child <- currChildren) {
+              if (isExtraRecord(schemaNode, child)) {
+                for (subChild <- child.children) toCopy += subChild
+              } else toCopy += child
+            }
+            currChildren = toCopy
+          }
+          schemaNode.children = currChildren.toList
+        }
+      case _ =>
+    }
+    // call this helper method on each of this Nodes's children.  There is no infinite recursion because
+    // eventually a sub-child will not have any more children.  This of course assumes that we are dealing
+    // with a rooted tree (there shouldn't be any cycles), which should be the case.
+    for (child <- schemaNode.children) removeExtraRecords(child)
+  }
+
+  /**
+   * Perform postProcessing; this happens *after* the TestNode tree is created but *before* that tree
+   * gets converted to a TestSchema
+   */
+  private def postProcessing(): Unit = {
+    removeExtraRecords(objectStack.head)
+  }
+
+  private def getSimpleType(schemaNode: TestNode): FieldType = {
+    if (!schemaNode.isSimple) throw new IllegalArgumentException(s"Node $schemaNode is not a simple type!")
+    schemaNode.fieldType
+  }
+
+  private def nodeToField(schemaNode: TestNode): TestField = {
+    // by default, if this node doesn't have a name, its data type is used as the field name.
+    // This should only ever be the case for anonymous choices.
+    val recordField: TestField = new TestField(
+      schemaNode.name.getOrElse(schemaNode.fieldType.toString),
+      schemaNodeToDataType(schemaNode)
+    )
+    if (schemaNode.isOptional) new OptionalTestField(recordField) else recordField
+  }
+
+  private def newRecordSchema(schemaNode: TestNode, fieldList: List[TestField]): TestSchema = {
+    new TestSchema(schemaNode.name.getOrElse(""), schemaNode.namespace, fieldList)
+  }
+
+  /**
+   * Helper method to specifically convert a TestNode known to be a Record into a TestSchema.
+   * Each Child of the TestNode becomes a Field of the TestSchema
+   * @param schemaNode the node to convert to a TestSchema
+   * @return a tuple containing the finalized TestSchema and its corresponding Record Data Type
+   */
+  private def schemaNodeToRecordType(schemaNode: TestNode): RecordDataType = {
+    val fieldList: ListBuffer[TestField] = ListBuffer()
+    for (child <- schemaNode.children) fieldList += nodeToField(child)
+    val TestSchema: TestSchema = newRecordSchema(schemaNode, fieldList.toList)
+    RecordDataType(TestSchema)
+  }
+
+  /**
+   * Helper method to convert a TestNode known to be a choice into a Choice data type.
+   * This is able to handle a DFDL schema in which either named elements are directly sub-members of
+   * DFDL choices or if they are embedded in another element (which corresponds to being in a Record).
+   * In the end, if they are not already inside a Record, then they are put there. Choices cannot
+   * have fields, only possible sub-types, so anything that would be a "field" has a wrapper Record
+   * put around it.
+   * @param schemaNode the node to convert to a Choice data type
+   * @return a Choice data type as described above
+   */
+  private def choiceNodeToChoiceSchema(schemaNode: TestNode): DataType = {
+    val childList: ListBuffer[DataType] = new ListBuffer()
+    for (child <- schemaNode.children) {
+      if (child.name.isEmpty) {
+        childList += schemaNodeToRecordType(child)
+      } else {
+        // In the case where the child has a name, we embed it in an additional TestSchema,
+        // since we must add some DataType rather than a Field.
+        val fieldList: ListBuffer[TestField] = ListBuffer()
+        fieldList += nodeToField(child)
+        val choiceOption: TestSchema = newRecordSchema(child, fieldList.toList)
+        childList += RecordDataType(choiceOption)
+      }
+    }
+    ChoiceDataType(childList.toList)
+  }
+
+  /**
+   * Local helper method to appropriately convert a TestNode into an appropriate Record
+   * Data Type.  Records and Choices get routed to other helper methods, and Arrays are handled in the method.
+   * @param schemaNode the node to convert to a Record Data Type
+   * @return the finalized Record Data type
+   */
+  private def schemaNodeToDataType(schemaNode: TestNode): DataType = {
+    schemaNode.fieldType match {
+      case ArrayType =>
+        if (schemaNode.isSimple) {
+          ArrayDataType(new DataType(getSimpleType(schemaNode)))
+        } else if (schemaNode.children.nonEmpty) {
+          ArrayDataType(schemaNodeToRecordType(schemaNode))
+        } else {
+          throw new IllegalStateException(
+            s"Array Node $schemaNode either needs to be simple or have child nodes!"
+          )
+        }
+      case RecordType => schemaNodeToRecordType(schemaNode)
+      case ChoiceType => choiceNodeToChoiceSchema(schemaNode)
+      case _ => schemaNode.simpleType.get
+    }
+  }
+
+  /**
+   * In general, this method can be used to extract the TestSchema node from the tuple result
+   * of schemaNodeToRecordSchema.  However, it is really only used to kick off the conversion from
+   * the root TestNode.
+   * @param schemaNode the node from which a TestSchema will be extracted
+   * @return the processed TestSchema represented by the input
+   */
+  private def getRecordSchema(schemaNode: TestNode): TestSchema = {
+    schemaNodeToRecordType(schemaNode).childSchema
+  }
+
+}
+
+/**
+ * The sole purpose of this Companion Object is to contain this one static variable.
+ */
+object RecordWalker {
+  /**
+   * Used to suppress all of the extra printing from running test cases
+   */
+  final val PRODUCTION_MODE: Boolean = true

Review comment:
       I'm concerend that this test code is fairly complex, and that it's pretty easy for there to be a small bug in here which could lead to us not realizing there's an issue with the walker. I'm wondering if there's a less complex way to test the DSOM walker to verify that it's doing the right thing?
   
   We really just want to validate that the DSOM walker creates the correct events in the correct order. We don't really need to use these events to build anything to test that it's walking correctly. Perhaps it would be simpler if the TestDSOMWalker just received an ordered list of expected events and properties of those events. As the TestDSOMWalker walks the tree and receives events, it just ensures that they match the expected list of events.
   
   There's probably multiple approaches, but the first thing that comes to my mind is to implement customer the walker.Views that have a custom comparision function to compare things we want to test. Note that we don't necessarily need to check all properties, for example, it might make sense to ignore group, groupMembers, simpleType, complexType, etc. since these are all implied correct if we get the correct events. So one of these might look look something like this:
   ```scala
   case class TestRootView(
     override isSimpleType: Boolean,
     override isComplexType: Boolean,
     ...)
     extends walker.RootView {
     
     override def equals(that: AnyRef): Boolean =
       that match {
         case rv: RootView =>
           this.isSimpleType == rv.isSimpletype &&
           this.isComplexType == rv.isComplexType &&
           ...
       case _ => false
     }
   }
   ```
   We'd have a concrete TestView for each walker View we could get from the walker. Then the TestDSOMWalker could look something like this:
   ```scala
   class TestDSOMWalker(expectedEvents: Seq[TestView]) {
     var events = expectedEvents
     def onSomeEvent(view: walker.SomeView): Unit = {
       val head :: tail = events
       assertTrue(head == view)
       events = tail 
     }
   }
   ```
   Then to test, this becomes:
   ```
   def testWalker1() = {
     val expectedEvents = Seq(
       TestRootView(....),
       TestComplexTypeView(...),
       TestSequenceView(...),
       ...
     )
     ...
     val testWalker = new TestDSOMWalker(expectedEvents)
     testWalker.walk(pf.rootView)
   }
   ```
   
   There's still a little complexity here--we need to create these concrete views and ensure we're testing the right properties, including start/end events. But I think it's a little easier to verify that the walker implementation is doing the right thing with a simpler approach.




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453978467



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,163 @@
+/*
+ * 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 org.apache.daffodil.dpath.NodeInfo.PrimType
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+  def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    walker.onTermEnd(this)
+  }
+}
+
+trait SimpleTypeView {
+  def primType: PrimType
+}
+trait ComplexTypeView {
+  def group: ModelGroupView
+}
+
+trait ModelGroupView extends TermView {
+  def groupMembers: Seq[TermView]
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    this.groupMembers.foreach(_.walkDSOM(walker))
+    walker.onTermEnd(this)
+  }
+}
+
+trait ChoiceView extends ModelGroupView
+trait SequenceView extends ModelGroupView
+trait GroupRefView extends ModelGroupView {
+  def isHidden: Boolean
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = if (!isHidden) super.walkDSOM(walker)
+}
+
+trait ElementDeclView extends CommonContextView {
+  def isSimpleType: Boolean
+  def isComplexType: Boolean
+  def simpleType: SimpleTypeView
+  def complexType: ComplexTypeView
+  def hasDefaultValue: Boolean
+}
+
+trait ElementBaseView extends ElementDeclView with TermView {
+  def name: String
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    if (isComplexType) {
+      walker.onTypeBegin(Right(complexType))
+      complexType.group.walkDSOM(walker)
+      walker.onTypeBegin(Right(complexType))
+    } else {
+      walker.onTypeBegin(Left(simpleType))
+      walker.onTypeBegin(Left(simpleType))
+    }
+    walker.onTermEnd(this)
+  }
+
+}
+
+trait RootView extends ElementBaseView
+
+/**
+ * A class designed to walk the internal representation of a DFDL Schema File.
+ *
+ * There are 2 main event handlers an implementing class has to worry about: one for Terms, and
+ * another for Types.  These are called as the DSOM is walked.
+ *
+ * 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.
+ * @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] {

Review comment:
       While I actually think Scala and Java Generics are pretty compatible, except for the fact that Scala supports things like Any, AnyRef, Unit, etc. while Java doesn't have those, I went ahead and removed the generic type.  It only makes things look slightly less clean for RecordWalker, but I think the fact its only purpose was to denote an awkward return type of one method it makes sense to take it out.




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



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

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455925365



##########
File path: daffodil-japi/src/main/scala/org/apache/daffodil/japi/Daffodil.scala
##########
@@ -373,6 +374,12 @@ class ProcessorFactory private[japi] (private var pf: SProcessorFactory)
     new DataProcessor(dp)
   }
 
+  /**
+   * Exposes the RootView object corresponding to this ProcessorFactory.  This can
+   * be used to start a walk using the walkFromRoot method in a DSOM Walker.
+   */
+  def rootView: RootView = pf.rootView

Review comment:
       Yeah, the submodule is really only needed when we want to include View stuff in public docs.
   
   Not being public does mean in order to use the walker/views you kindof have to know what you're doing and know where to find these Views. But that's not unreasonable for experimental things. Agree that the subprojet probably isn't needed right now.




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453838690



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,163 @@
+/*
+ * 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 org.apache.daffodil.dpath.NodeInfo.PrimType
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+  def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {

Review comment:
       The walk is currently initiated here:
   ```scala
   final def walkFromRoot(schemaSetRoot: RootView): T = {
       onWalkBegin(schemaSetRoot)
       schemaSetRoot.walkDSOM(this)
       // onWalkEnd will return the value returned by this method!
       onWalkEnd(schemaSetRoot)
     }
   ```
   As you can see, this method requires `walkDSOM` to be a public member of `TermView`.  I am not quite sure how exactly the walk would begin without this method being public.




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455887835



##########
File path: daffodil-core/src/test/scala/org/apache/daffodil/dsom/walker/TestDSOMWalker.scala
##########
@@ -0,0 +1,229 @@
+/*
+ * 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 org.apache.daffodil.util._
+import org.apache.daffodil.compiler.{ Compiler, ProcessorFactory }
+import org.junit.Test
+import org.junit.Assert._
+
+class TestDSOMWalker {
+
+  @Test def testComplexTypes(): Unit = {
+    val testSchema = SchemaUtils.dfdlTestSchema(
+      <xs:include schemaLocation="org/apache/daffodil/xsd/DFDLGeneralFormat.dfdl.xsd"/>,
+      <dfdl:format ref="ex:GeneralFormat"
+                   alignment="implicit" alignmentUnits="bits" occursCountKind="implicit"
+                   lengthKind="delimited" encoding="ASCII"/>,
+      <xs:element name="PersonData">
+        <xs:complexType>
+          <xs:sequence>
+            <xs:choice>
+              <xs:element name="age" type="xs:int" minOccurs="1" maxOccurs="1"/>
+            </xs:choice>
+            <xs:group ref="testGroup" />
+          </xs:sequence>
+        </xs:complexType>
+      </xs:element>
+      <xs:group name="testGroup">
+        <xs:sequence />
+      </xs:group>
+    )
+    val pf: ProcessorFactory = Compiler().compileNode(testSchema)
+    assertEquals(s"This basic Schema: $testSchema should compile; here are some diagnostics: ${pf.getDiagnostics}", false, pf.isError)
+    val walker: BasicWalker = new BasicWalker()
+    walker.walkFromRoot(pf.rootView)
+    val nodeStack: List[Either[TermView, TypeView]] = walker.nodeArr.toList
+    assertEquals(s"Node Stack $nodeStack did not have the expected number of elements", 7, nodeStack.size)
+    nodeStack.head match {
+      case Left(personElement: RootView) =>
+        assertEquals("The root element should be named 'PersonData'", "PersonData", personElement.name)
+      case Left(someClass) => fail(s"The Root element was not of type RootView, it was a ${someClass.getClass}!")
+      case Right(someClass) => fail(s"The Root element was not of type RootView, it was a ${someClass.getClass}!")
+    }
+    nodeStack(1) match {
+      case Right(_: ComplexTypeView) =>
+      case Left(someClass) => fail(s"The Root element did not contain a complexType wrapper child, it was a ${someClass.getClass}!")
+      case Right(someClass) => fail(s"The Root element element did not contain a complexType wrapper child, it was a ${someClass.getClass}!")
+    }
+    nodeStack(2) match {
+      case Left(_: SequenceView) =>
+      case Left(someClass) => fail(s"The complexType element did not contain a Sequence child, it was a ${someClass.getClass}!")
+      case Right(someClass) => fail(s"The complexType element did not contain a Sequence child, it was a ${someClass.getClass}!")
+    }
+    nodeStack(3) match {
+      case Left(_: ChoiceView) =>
+      case Left(someClass) => fail(s"The Sequence element did not contain a Choice child, it was a ${someClass.getClass}!")
+      case Right(someClass) => fail(s"The Sequence element did not contain a Choice child, it was a ${someClass.getClass}!")
+    }
+    nodeStack(4) match {
+      case Left(nameElement: ElementBaseView) =>
+        assertEquals("The second child element should be named 'age'", "age", nameElement.name)
+        assertEquals("The second child element should be a simple type", true, nameElement.isSimpleType)
+      case Left(someClass) => fail(s"The Sequence element did not contain a second Element child, it was a ${someClass.getClass}!")
+      case Right(someClass) => fail(s"The Sequence element did not contain a second Element child, it was a ${someClass.getClass}!")
+    }
+    nodeStack(5) match {
+      case Right(simpleType: SimpleTypeView) =>
+        simpleType.primType match {
+          case _:IntView =>
+          case _ => fail(s"The 'age' element should be of simple type Int")
+        }
+      case Left(someClass) => fail(s"The 'age' element did not have a SimpleTypeView, it was a ${someClass.getClass}!")
+      case Right(someClass) => fail(s"The 'age' element did not have a SimpleTypeView, it was a ${someClass.getClass}!")
+    }
+    nodeStack(6) match {
+      case Left(_: GroupRefView) =>
+      case Left(someClass) => fail(s"The Sequence element did not contain a GroupRef child, it was a ${someClass.getClass}!")
+      case Right(someClass) => fail(s"The Sequence element did not contain a GroupRef child, it was a ${someClass.getClass}!")
+    }
+  }

Review comment:
       I agree that it is robust, and I had it this way mainly for the sake of pattern matching and being able to figure out types of specific View elements on the stack when tests were failing.  That being said, for simplicity sake, I think the series of asserts based on `isInstanceOf` and `asInstanceOf` would be best.  I tried implementing a case class with an equals method that takes in an `AnyRef` as an argument as you suggested in an earlier comment, and it turned out to be an even longer solution.




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453889437



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/SimpleTypes.scala
##########
@@ -53,9 +54,9 @@ trait TypeBase {
 trait NonPrimTypeMixin
 
 sealed trait SimpleTypeBase extends TypeBase
-  with HasOptRepTypeMixin {
+  with HasOptRepTypeMixin with SimpleTypeView {
 
-  def primType: PrimType
+  override def primType: PrimType

Review comment:
       You mentioned that `PrimType` is changed frequently - would this list pose an issue because of that? i.e. wouldn't future developers have to go change this list of primitive Views every time `PrimType` is changed?
   
   I definitely agree though, it would be ideal to design a system where no internal classes would have to be exposed at all.




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



[GitHub] [incubator-daffodil] stevedlawrence merged pull request #394: Feature Addition: API for traversing DSOM and generating NIFI RecordSchemas from DSOM

Posted by GitBox <gi...@apache.org>.
stevedlawrence merged pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394


   


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



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

Posted by GitBox <gi...@apache.org>.
mbeckerle commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453848513



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,163 @@
+/*
+ * 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 org.apache.daffodil.dpath.NodeInfo.PrimType
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+  def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    walker.onTermEnd(this)
+  }
+}
+
+trait SimpleTypeView {
+  def primType: PrimType
+}
+trait ComplexTypeView {
+  def group: ModelGroupView
+}
+
+trait ModelGroupView extends TermView {
+  def groupMembers: Seq[TermView]
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    this.groupMembers.foreach(_.walkDSOM(walker))
+    walker.onTermEnd(this)
+  }
+}
+
+trait ChoiceView extends ModelGroupView
+trait SequenceView extends ModelGroupView
+trait GroupRefView extends ModelGroupView {
+  def isHidden: Boolean
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = if (!isHidden) super.walkDSOM(walker)
+}
+
+trait ElementDeclView extends CommonContextView {
+  def isSimpleType: Boolean
+  def isComplexType: Boolean
+  def simpleType: SimpleTypeView
+  def complexType: ComplexTypeView
+  def hasDefaultValue: Boolean
+}
+
+trait ElementBaseView extends ElementDeclView with TermView {
+  def name: String
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    if (isComplexType) {
+      walker.onTypeBegin(Right(complexType))
+      complexType.group.walkDSOM(walker)
+      walker.onTypeBegin(Right(complexType))
+    } else {
+      walker.onTypeBegin(Left(simpleType))
+      walker.onTypeBegin(Left(simpleType))
+    }
+    walker.onTermEnd(this)
+  }
+
+}
+
+trait RootView extends ElementBaseView
+
+/**
+ * A class designed to walk the internal representation of a DFDL Schema File.
+ *
+ * There are 2 main event handlers an implementing class has to worry about: one for Terms, and
+ * another for Types.  These are called as the DSOM is walked.
+ *
+ * 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.
+ * @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] {
+
+  /**

Review comment:
       While we do want stability, I think we should separate that from having a need for Java support. This particular API is so close to the core of the Daffodil schema compiler that I think requiring it to be scala only is fine. I don't think it is necessary for people to be able to write extensions that walk the internals parse trees in Java. 
   
   It's not that this is not possible, it's just something I think adds little value for us to support, at least for now with this being the very first Daffodil "metadata integration". 




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



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

Posted by GitBox <gi...@apache.org>.
mbeckerle commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455923853



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,177 @@
+/*
+ * 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
+
+/**
+ * Below is a series of "View" Mixins defined such that particular simple attributes
+ * of the core DSOM classes are exposed as a Walk is performed.
+ *
+ * Several of them have a walkDSOM method.  By default, this method is called as the walk is performed,
+ * but a user implementing AbstractDSOMWalker can override this behavior by overriding walkFromRoot
+ * and implementing their own event handlers for each of the DSOM types.
+ */
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+}
+
+trait TypeView
+trait SimpleTypeView extends TypeView {
+  def primType: PrimTypeView
+}
+trait ComplexTypeView extends TypeView {
+  def group: ModelGroupView
+}
+
+trait ModelGroupView extends TermView {
+  def groupMembers: Seq[TermView]
+}
+
+trait ChoiceView extends ModelGroupView
+trait SequenceView extends ModelGroupView
+trait GroupRefView extends ModelGroupView {
+  def isHidden: Boolean
+}
+
+trait ElementDeclView extends CommonContextView {
+  def isSimpleType: Boolean
+  def isComplexType: Boolean
+  def simpleType: SimpleTypeView
+  def complexType: ComplexTypeView
+  def hasDefaultValue: Boolean

Review comment:
       If f NiFi doesn't need it, definitely leave it off. It's pretty good evidence that perhaps nobody will need it, and every thing we put on these views will cause someone to wonder whether or not they *need to be using it*. 




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



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

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r454291104



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,163 @@
+/*
+ * 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 org.apache.daffodil.dpath.NodeInfo.PrimType
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+  def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {

Review comment:
       Seems reasonable. So it sounds like basically the API becomes all the different Views and their members, plus some new public method to get the RootView. WIth the RootView, the user can traverse the DSOM tree however they want. And then we have the recursive DSOM walker implementation for users that just want to get a bunch of events and not have to worry about how things DSOM is walked. This feel like it makes the API a little larger and could make it difficult to change in the future, but as long as its experimental until it seems stable that's fine with me.




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455460831



##########
File path: daffodil-core/src/test/resources/test/standard-xsd/bmp.xsd
##########
@@ -0,0 +1,656 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--

Review comment:
       Yes, I noticed that - this paradigm makes more sense than what I had.  In the JUnit-based test class in the latest revision, the schemas are in the code and passed to the helper "dfdlTestSchema" I saw being used in the other test cases.  Let me know if you think the one I made for SimpleTypes is too long for a Unit Test environment; I could always break up each SimpleType into its own test




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453865556



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,163 @@
+/*
+ * 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 org.apache.daffodil.dpath.NodeInfo.PrimType
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+  def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    walker.onTermEnd(this)
+  }
+}
+
+trait SimpleTypeView {
+  def primType: PrimType
+}
+trait ComplexTypeView {
+  def group: ModelGroupView
+}
+
+trait ModelGroupView extends TermView {
+  def groupMembers: Seq[TermView]
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    this.groupMembers.foreach(_.walkDSOM(walker))
+    walker.onTermEnd(this)
+  }
+}
+
+trait ChoiceView extends ModelGroupView
+trait SequenceView extends ModelGroupView
+trait GroupRefView extends ModelGroupView {
+  def isHidden: Boolean
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = if (!isHidden) super.walkDSOM(walker)
+}
+
+trait ElementDeclView extends CommonContextView {
+  def isSimpleType: Boolean
+  def isComplexType: Boolean
+  def simpleType: SimpleTypeView
+  def complexType: ComplexTypeView
+  def hasDefaultValue: Boolean
+}
+
+trait ElementBaseView extends ElementDeclView with TermView {
+  def name: String
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    if (isComplexType) {
+      walker.onTypeBegin(Right(complexType))
+      complexType.group.walkDSOM(walker)
+      walker.onTypeBegin(Right(complexType))
+    } else {
+      walker.onTypeBegin(Left(simpleType))
+      walker.onTypeBegin(Left(simpleType))
+    }
+    walker.onTermEnd(this)
+  }
+
+}
+
+trait RootView extends ElementBaseView
+
+/**
+ * A class designed to walk the internal representation of a DFDL Schema File.
+ *
+ * There are 2 main event handlers an implementing class has to worry about: one for Terms, and
+ * another for Types.  These are called as the DSOM is walked.
+ *
+ * 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.
+ * @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] {
+
+  /**

Review comment:
       It is possible to build nars for projects that use scala code, I previously had all of this code in a maven project that mixed NIFI-specific Java code and Scala code.  It only required a few additional dependencies to include.
   
   That being said, if the NiFi developer doesn't have to code in Scala, they don't have to worry about any of that - they would only need to do this configuration if they wanted to implement AbstractDSOMWalker in Scala




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r454720851



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,163 @@
+/*
+ * 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 org.apache.daffodil.dpath.NodeInfo.PrimType
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+  def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {

Review comment:
       This would make the API larger due to this extra flexibility, I agree.  I went ahead and removed the `final` modifier from this method in the latest revision.




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455460831



##########
File path: daffodil-core/src/test/resources/test/standard-xsd/bmp.xsd
##########
@@ -0,0 +1,656 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--

Review comment:
       That paradigm makes more sense than what I had.  In the JUnit-based test class in the latest revision, the schemas are in the code and passed to the helper "dfdlTestSchema" I saw being used in the other test cases.  Let me know if you think the one I made for SimpleTypes is too long for a Unit Test environment; I could always break up each SimpleType into its own test




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



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

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453892222



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/SimpleTypes.scala
##########
@@ -53,9 +54,9 @@ trait TypeBase {
 trait NonPrimTypeMixin
 
 sealed trait SimpleTypeBase extends TypeBase
-  with HasOptRepTypeMixin {
+  with HasOptRepTypeMixin with SimpleTypeView {
 
-  def primType: PrimType
+  override def primType: PrimType

Review comment:
       I think list of PrimTypes isn't changed that often (and likely won't change anytime soon). The things that do change relatively frequently is the functions defined in the prim types. Because these change so often, we might not want those members to be publicly visible to a dsom walker. The View trait allows us to decide which members are made visible, which I suspect is not going to be many, if any at all. So you can still match on thes prim type views to figure out the type.
   
   You are correct that if a new PrimType is added, we'll also need to add a PrimTypeView as well. Not much we can do about that though.




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453865556



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,163 @@
+/*
+ * 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 org.apache.daffodil.dpath.NodeInfo.PrimType
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+  def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    walker.onTermEnd(this)
+  }
+}
+
+trait SimpleTypeView {
+  def primType: PrimType
+}
+trait ComplexTypeView {
+  def group: ModelGroupView
+}
+
+trait ModelGroupView extends TermView {
+  def groupMembers: Seq[TermView]
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    this.groupMembers.foreach(_.walkDSOM(walker))
+    walker.onTermEnd(this)
+  }
+}
+
+trait ChoiceView extends ModelGroupView
+trait SequenceView extends ModelGroupView
+trait GroupRefView extends ModelGroupView {
+  def isHidden: Boolean
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = if (!isHidden) super.walkDSOM(walker)
+}
+
+trait ElementDeclView extends CommonContextView {
+  def isSimpleType: Boolean
+  def isComplexType: Boolean
+  def simpleType: SimpleTypeView
+  def complexType: ComplexTypeView
+  def hasDefaultValue: Boolean
+}
+
+trait ElementBaseView extends ElementDeclView with TermView {
+  def name: String
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    if (isComplexType) {
+      walker.onTypeBegin(Right(complexType))
+      complexType.group.walkDSOM(walker)
+      walker.onTypeBegin(Right(complexType))
+    } else {
+      walker.onTypeBegin(Left(simpleType))
+      walker.onTypeBegin(Left(simpleType))
+    }
+    walker.onTermEnd(this)
+  }
+
+}
+
+trait RootView extends ElementBaseView
+
+/**
+ * A class designed to walk the internal representation of a DFDL Schema File.
+ *
+ * There are 2 main event handlers an implementing class has to worry about: one for Terms, and
+ * another for Types.  These are called as the DSOM is walked.
+ *
+ * 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.
+ * @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] {
+
+  /**

Review comment:
       It is possible to build nars for projects that use scala code, I previously had all of this code in a maven project that mixed NIFI-specific Java code and Scala code.  It only required a few additional dependencies.
   
   That being said, if the NiFi developer doesn't have to code in Scala, they don't have to worry about any of that - they would only need to do this configuration if they wanted to implement AbstractDSOMWalker in Scala




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453982477



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/SimpleTypes.scala
##########
@@ -53,9 +54,9 @@ trait TypeBase {
 trait NonPrimTypeMixin
 
 sealed trait SimpleTypeBase extends TypeBase
-  with HasOptRepTypeMixin {
+  with HasOptRepTypeMixin with SimpleTypeView {
 
-  def primType: PrimType
+  override def primType: PrimType

Review comment:
       I went ahead and defined an Enum type called PrimTypeView and created Enum representations for all the "PrimType" enum types within it.  I could not find a way to make the NodeInfo class import *anything* from `dsom.walker`, so I was unable to make it extend any trait I defined there.  Instead, I added a new field to `SimpleTypeBase` called `primTypeView` which just matches `primType` to my new Enum type.  Let me know if you think this approach should be changed; the pattern matching is done in `SimpleTypeBase` itself.




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453127490



##########
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:
       That makes sense!  I use that approach in the latest commit.




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



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

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453973431



##########
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:
       In theory, the only thing that needs to be in the Daffodil codebase is the API, which is essentially the AbstractDSOMWalker and all the Views traits. Nothing depending on NiFi should go in daffodil-core because that means core depends on NiFi and thus anyone using Daffodil now needs a NiFi jar--the use of Daffodli shouldn't depend on NiFi. 
   
   I don't think it really matters where the AbstractDSOMWalker and View traits go. They can probably just stay in core. Note that they can't go in the japi/sapi directories. This is because daffodilxXapi currently depends on daffodil-core (xapi is essentially a basic wrapper around core). If these AbstractDSOMWalker and View traits were in the daffodil-xapi, then daffodil-core would then depend on daffodil-xapi, so we'd have a circular dependency. So the AbstractDSOMWalker and View traits either stay in daffodil-core or in a new subproject. Since we are considering this experiemental, maybe it makes sense to keep it in core, and when we finalize an API and move it from experimental, we can move it to a separate dir where we can generate public java/scaladocs? I guess this is essentially what we've done with daffodil-udf. I consider it a public API so it makes sense to get it's own directory where we can generate docs for it.
   
   So where does everything go that uses NiFi? It should probably all stick together in one place. Perhaps that place right now is the nifi-daffodil repo, just to make it very obvious what is NiFi and what is Daffodil, and to highlight what changes are Daffodil API additions and what are NiFi usages of this API? Once that's finalized and the nifi-daffodil repo is in a stable state and the experimental API is considered sufficient, it then becomes pretty trivial to either merge that into Nifi if they are willing to accept it, or merge it into a new Daffodli "integrations" directory for it to live. Or it could just remain separate--though I'd prefer it get merged somewhere.




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



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

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r456009020



##########
File path: daffodil-sapi/src/main/scala/org/apache/daffodil/sapi/Daffodil.scala
##########
@@ -338,6 +339,13 @@ class ProcessorFactory private[sapi] (private var pf: SProcessorFactory)
     new DataProcessor(dp)
   }
 
+  class ExperimentalWrapper(val rootView: RootView)
+  /**
+   * Exposes the RootView object corresponding to this ProcessorFactory.  This can
+   * be used to start a walk using the walkFromRoot method in a DSOM Walker.
+   */
+  val experimental: ExperimentalWrapper = new ExperimentalWrapper(pf.rootView)
+

Review comment:
       I think it's reasonable to use the ``object experimental`` technique for the scala api. I think that makes fora cleaner scaladoc and feels more natural to scala users.




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453127329



##########
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:
       I agree, that's the more appropriate location.  I might get rid of this flag eventually, the only reason it exists is if the Unit tests fail you can turn it on and see what the data structures look like at each stage of the processing.




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453838690



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,163 @@
+/*
+ * 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 org.apache.daffodil.dpath.NodeInfo.PrimType
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+  def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {

Review comment:
       The walk is currently initiated here:
   ```scala
   final def walkFromRoot(schemaSetRoot: RootView): T = {
     onWalkBegin(schemaSetRoot)
     schemaSetRoot.walkDSOM(this)
     // onWalkEnd will return the value returned by this method!
     onWalkEnd(schemaSetRoot)
   }
   ```
   As you can see, this method requires `walkDSOM` to be a public member of `TermView`.  I am not quite sure how exactly the walk would begin without this method being public.




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r454722287



##########
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:
       It definitely makes sense to keep the API and any NiFi-related/dependent code separate; in the latest revision I removed RecordWalker and RecordSchemaNode from core and made some general classes for testing to replace them.  I think it would make the most sense to move both of these to the nifi-daffodil repo, but as it currently stands they're both written in Scala and I'm not sure if there's necessarily a 1-1 Java conversion.




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



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

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455801271



##########
File path: daffodil-core/src/test/scala/org/apache/daffodil/dsom/walker/TestDSOMWalker.scala
##########
@@ -0,0 +1,229 @@
+/*
+ * 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 org.apache.daffodil.util._
+import org.apache.daffodil.compiler.{ Compiler, ProcessorFactory }
+import org.junit.Test
+import org.junit.Assert._
+
+class TestDSOMWalker {
+
+  @Test def testComplexTypes(): Unit = {
+    val testSchema = SchemaUtils.dfdlTestSchema(
+      <xs:include schemaLocation="org/apache/daffodil/xsd/DFDLGeneralFormat.dfdl.xsd"/>,
+      <dfdl:format ref="ex:GeneralFormat"
+                   alignment="implicit" alignmentUnits="bits" occursCountKind="implicit"
+                   lengthKind="delimited" encoding="ASCII"/>,
+      <xs:element name="PersonData">
+        <xs:complexType>
+          <xs:sequence>
+            <xs:choice>
+              <xs:element name="age" type="xs:int" minOccurs="1" maxOccurs="1"/>
+            </xs:choice>
+            <xs:group ref="testGroup" />
+          </xs:sequence>
+        </xs:complexType>
+      </xs:element>
+      <xs:group name="testGroup">
+        <xs:sequence />
+      </xs:group>
+    )
+    val pf: ProcessorFactory = Compiler().compileNode(testSchema)
+    assertEquals(s"This basic Schema: $testSchema should compile; here are some diagnostics: ${pf.getDiagnostics}", false, pf.isError)
+    val walker: BasicWalker = new BasicWalker()
+    walker.walkFromRoot(pf.rootView)
+    val nodeStack: List[Either[TermView, TypeView]] = walker.nodeArr.toList
+    assertEquals(s"Node Stack $nodeStack did not have the expected number of elements", 7, nodeStack.size)
+    nodeStack.head match {
+      case Left(personElement: RootView) =>
+        assertEquals("The root element should be named 'PersonData'", "PersonData", personElement.name)
+      case Left(someClass) => fail(s"The Root element was not of type RootView, it was a ${someClass.getClass}!")
+      case Right(someClass) => fail(s"The Root element was not of type RootView, it was a ${someClass.getClass}!")
+    }
+    nodeStack(1) match {
+      case Right(_: ComplexTypeView) =>
+      case Left(someClass) => fail(s"The Root element did not contain a complexType wrapper child, it was a ${someClass.getClass}!")
+      case Right(someClass) => fail(s"The Root element element did not contain a complexType wrapper child, it was a ${someClass.getClass}!")
+    }
+    nodeStack(2) match {
+      case Left(_: SequenceView) =>
+      case Left(someClass) => fail(s"The complexType element did not contain a Sequence child, it was a ${someClass.getClass}!")
+      case Right(someClass) => fail(s"The complexType element did not contain a Sequence child, it was a ${someClass.getClass}!")
+    }
+    nodeStack(3) match {
+      case Left(_: ChoiceView) =>
+      case Left(someClass) => fail(s"The Sequence element did not contain a Choice child, it was a ${someClass.getClass}!")
+      case Right(someClass) => fail(s"The Sequence element did not contain a Choice child, it was a ${someClass.getClass}!")
+    }
+    nodeStack(4) match {
+      case Left(nameElement: ElementBaseView) =>
+        assertEquals("The second child element should be named 'age'", "age", nameElement.name)
+        assertEquals("The second child element should be a simple type", true, nameElement.isSimpleType)
+      case Left(someClass) => fail(s"The Sequence element did not contain a second Element child, it was a ${someClass.getClass}!")
+      case Right(someClass) => fail(s"The Sequence element did not contain a second Element child, it was a ${someClass.getClass}!")
+    }
+    nodeStack(5) match {
+      case Right(simpleType: SimpleTypeView) =>
+        simpleType.primType match {
+          case _:IntView =>
+          case _ => fail(s"The 'age' element should be of simple type Int")
+        }
+      case Left(someClass) => fail(s"The 'age' element did not have a SimpleTypeView, it was a ${someClass.getClass}!")
+      case Right(someClass) => fail(s"The 'age' element did not have a SimpleTypeView, it was a ${someClass.getClass}!")
+    }
+    nodeStack(6) match {
+      case Left(_: GroupRefView) =>
+      case Left(someClass) => fail(s"The Sequence element did not contain a GroupRef child, it was a ${someClass.getClass}!")
+      case Right(someClass) => fail(s"The Sequence element did not contain a GroupRef child, it was a ${someClass.getClass}!")
+    }
+  }

Review comment:
       These feels a little verbose to me. All the fail messages are perhaps helpful when a test fails, but that should be rare, and it makes it difficult to visually determine that the tests are checking the right things, which I'd argue is more important.
   
   I think part of the problem this looks so verbose is the the node stack being an Either, so you need to do a bunch of left/right checks. But that doesn't give us much. You still need to do type matcing inside the Left/Right, so the Left/Right just adds a layer of indirection. Instead, you can just make nodeArray be an Array[AnyRef].
   
   Then, if we removed all the matching and messages, the above can be condensed down to something like this:
   ```scala
   assertEquals(7, nodeStack.size)
   assertTrue(nodeStack(0).isInstanceOf[RootView])
   assertEquals(nodeStack(0).asInstanceOf[RootView].name, "PersonData")
   assertTrue(nodeStack(1).isInstanceOf[ComplexTypeView])
   assertTrue(nodeStack(2).isInstanceOf[SequenceView])
   assertTrue(nodeStack(3).isInstanceOf[ChoiceView])
   assertTrue(nodeStack(4).isInstanceOf[ElementBaseView])
   assertEquals(nodeStack(4).asInstanceOf[ElementBaseView].name, "age")
   assertEquals(nodeStack(4).asInstanceOf[ElementBaseView].isSimpleType, true)
   assertTrue(nodeStack(5).isInstanceOf[SimpleTypeView])
   assertTrue(nodeStack(5).asInstanceOf[SimpleTypeView].primType.isInstanceOf[IntView])
   assertTrue(nodeStack(6).isInstanceOf[GroupRefView])
   ```
   Which I think is much easier to visually confirm that we're checking the right things. The suggestion I previously made about having concrete implemenations of the Views and an equal function make these checks a bit more automated and maybe a bit more clear (probably gets rid of all the instance of stuff), at the expense of needing a bit more code. But something more along the lines of this doesn't seem unreasonable to me.
   
   Note that this is much less scala-y, but readability often trumps that, especially when it comes to tests.




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



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

Posted by GitBox <gi...@apache.org>.
mbeckerle commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455922667



##########
File path: daffodil-japi/src/main/scala/org/apache/daffodil/japi/Daffodil.scala
##########
@@ -373,6 +374,12 @@ class ProcessorFactory private[japi] (private var pf: SProcessorFactory)
     new DataProcessor(dp)
   }
 
+  /**
+   * Exposes the RootView object corresponding to this ProcessorFactory.  This can
+   * be used to start a walk using the walkFromRoot method in a DSOM Walker.
+   */
+  def rootView: RootView = pf.rootView

Review comment:
       I think a separate submodule is going too far unless that's necessary someday to grab the scaladoc from this and incorporate it into the public-facing scaladoc set. 
   I like the experimental object technique you showed. That's sufficient to me for now. 




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



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

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453842075



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,163 @@
+/*
+ * 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 org.apache.daffodil.dpath.NodeInfo.PrimType
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+  def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {

Review comment:
       Ah, that's right. I guess the only way to do it would be to have the walkDSOM function not be on the View but be on the actual DSOM SchemaComponents, and all the walking occurs on the internal/private/unstable API. Unclear if that adds much benefit though. Perhaps there's value in allowing the implementation to change how the DSOM is walked.




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



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

Posted by GitBox <gi...@apache.org>.
mbeckerle commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453850799



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,163 @@
+/*
+ * 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 org.apache.daffodil.dpath.NodeInfo.PrimType
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+  def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {

Review comment:
       There are different walk patterns yes. So I think keeping the walkers on the traits where they can be overridden or new ones added makes sense. 




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



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

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455042095



##########
File path: project/Dependencies.scala
##########
@@ -29,6 +29,7 @@ object Dependencies {
     "xml-resolver" % "xml-resolver" % "1.2",
     "commons-io" % "commons-io" % "2.6",
     "jline" % "jline" % "2.14.6",
+    "org.scalatest" %% "scalatest" % "3.2.0" % "test"

Review comment:
       I think it would be preferrable if these tests were changed to junit just so that all of our test uses the same 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



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

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455010022



##########
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:
       Re: the nifi-daffodil repo, if you're able to modify the nifi-daffodil repo to support scala, it should work fine in theory. I suspect the new controllers are mostly separate from the existing processors?
   
   Re: walkDSOM on ProcessorFactory, I think after discussions the consensus is that while a recursive walk is useful, there may be cases where it isn't sufficient. So I would suggust that we move all the ``walkDSOM`` functions off of the Views and onto the AbstractDSOMWalker (just like you had originally), so the AbstractDSOM walker has all the logic for how to recursively walk a DSOM. Then the only new function need on the the ProcessorFactory is an ability to get the RootView. If one wants to traverse this RootView manually, they can do so. But if they just want to traverse it recrusively and get events, they can create a concrete implementation of the AbstractDSOMWalker and implement those events (and potential override the walkDSOM function if they want. So for example:
   ```scala
   val rootView = processorFactory.rootView
   val walker = new DSOMWalkerImpl()
   walker.walkDSOM(rootView)
   ```
   So the View's don't contain any logic about how they should be traversed, since they don't really know that. Recurisve makes sense, but presumable there are reasonable traversals. And technically the AbstractDSOMWalker isn't even neccessary--the views have all the information needed to traverse them. But to be helpful, it makes sense to also provide a AbstractDSOMWalker interface that contains all the logic for traversing recursively so users don't have to think about that and can just think of it as a series of events.
   
   So this is very similar to what you originally started with, but just with Views to hide internal details of our dsom objects.




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



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

Posted by GitBox <gi...@apache.org>.
mbeckerle commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453864443



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,163 @@
+/*
+ * 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 org.apache.daffodil.dpath.NodeInfo.PrimType
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+  def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    walker.onTermEnd(this)
+  }
+}
+
+trait SimpleTypeView {
+  def primType: PrimType
+}
+trait ComplexTypeView {
+  def group: ModelGroupView
+}
+
+trait ModelGroupView extends TermView {
+  def groupMembers: Seq[TermView]
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    this.groupMembers.foreach(_.walkDSOM(walker))
+    walker.onTermEnd(this)
+  }
+}
+
+trait ChoiceView extends ModelGroupView
+trait SequenceView extends ModelGroupView
+trait GroupRefView extends ModelGroupView {
+  def isHidden: Boolean
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = if (!isHidden) super.walkDSOM(walker)
+}
+
+trait ElementDeclView extends CommonContextView {
+  def isSimpleType: Boolean
+  def isComplexType: Boolean
+  def simpleType: SimpleTypeView
+  def complexType: ComplexTypeView
+  def hasDefaultValue: Boolean
+}
+
+trait ElementBaseView extends ElementDeclView with TermView {
+  def name: String
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    if (isComplexType) {
+      walker.onTypeBegin(Right(complexType))
+      complexType.group.walkDSOM(walker)
+      walker.onTypeBegin(Right(complexType))
+    } else {
+      walker.onTypeBegin(Left(simpleType))
+      walker.onTypeBegin(Left(simpleType))
+    }
+    walker.onTermEnd(this)
+  }
+
+}
+
+trait RootView extends ElementBaseView
+
+/**
+ * A class designed to walk the internal representation of a DFDL Schema File.
+ *
+ * There are 2 main event handlers an implementing class has to worry about: one for Terms, and
+ * another for Types.  These are called as the DSOM is walked.
+ *
+ * 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.
+ * @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] {
+
+  /**

Review comment:
       Expermental until we have a second workable metadata integration sounds like the right thing. 




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



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

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453795846



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,163 @@
+/*
+ * 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 org.apache.daffodil.dpath.NodeInfo.PrimType
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+  def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    walker.onTermEnd(this)
+  }
+}
+
+trait SimpleTypeView {
+  def primType: PrimType
+}
+trait ComplexTypeView {
+  def group: ModelGroupView
+}
+
+trait ModelGroupView extends TermView {
+  def groupMembers: Seq[TermView]
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    this.groupMembers.foreach(_.walkDSOM(walker))
+    walker.onTermEnd(this)
+  }
+}
+
+trait ChoiceView extends ModelGroupView
+trait SequenceView extends ModelGroupView
+trait GroupRefView extends ModelGroupView {
+  def isHidden: Boolean
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = if (!isHidden) super.walkDSOM(walker)
+}
+
+trait ElementDeclView extends CommonContextView {
+  def isSimpleType: Boolean
+  def isComplexType: Boolean
+  def simpleType: SimpleTypeView
+  def complexType: ComplexTypeView
+  def hasDefaultValue: Boolean
+}
+
+trait ElementBaseView extends ElementDeclView with TermView {
+  def name: String
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    if (isComplexType) {
+      walker.onTypeBegin(Right(complexType))
+      complexType.group.walkDSOM(walker)
+      walker.onTypeBegin(Right(complexType))
+    } else {
+      walker.onTypeBegin(Left(simpleType))
+      walker.onTypeBegin(Left(simpleType))
+    }
+    walker.onTermEnd(this)
+  }
+
+}
+
+trait RootView extends ElementBaseView
+
+/**
+ * A class designed to walk the internal representation of a DFDL Schema File.
+ *
+ * There are 2 main event handlers an implementing class has to worry about: one for Terms, and
+ * another for Types.  These are called as the DSOM is walked.
+ *
+ * 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.
+ * @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] {
+
+  /**

Review comment:
       Regarding the *View stuff, this seems reasonable to me. One concern is that we probably want this to be a public stable API, that works in both Java and Scala. I *think* Scala traits get turned into interfaces that a Java class can access, but I'm not 100% sure. If we generate JavaDocs, which we probably want to do, I'm also not sure if the Scala traits turn into something useful that makes sense for a java API. Note that we use the genjavadocs plugin to generate javadocs for daffodil-japi. I'm wondering if we want to do something similar? So maybe we want to move all the walker related things that are part of the public API (i.e. everything in the dsom.walker package) to a daffodil-dsom-walker subproject? Then we can generate JavaDocs for that just like with do with the Daffodil JAPI.
   
   Related, I'm looking into [DAFFODIL-2368](https://issues.apache.org/jira/browse/DAFFODIL-2368), which is related to how we can merge docs from multiple subprojects. I think looking at the Javadocs that results from this would be a clear way to view what parts of this API are public and what aren't.

##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,163 @@
+/*
+ * 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 org.apache.daffodil.dpath.NodeInfo.PrimType
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+  def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    walker.onTermEnd(this)
+  }
+}
+
+trait SimpleTypeView {
+  def primType: PrimType
+}
+trait ComplexTypeView {
+  def group: ModelGroupView
+}
+
+trait ModelGroupView extends TermView {
+  def groupMembers: Seq[TermView]
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    this.groupMembers.foreach(_.walkDSOM(walker))
+    walker.onTermEnd(this)
+  }
+}
+
+trait ChoiceView extends ModelGroupView
+trait SequenceView extends ModelGroupView
+trait GroupRefView extends ModelGroupView {
+  def isHidden: Boolean
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = if (!isHidden) super.walkDSOM(walker)
+}
+
+trait ElementDeclView extends CommonContextView {
+  def isSimpleType: Boolean
+  def isComplexType: Boolean
+  def simpleType: SimpleTypeView
+  def complexType: ComplexTypeView
+  def hasDefaultValue: Boolean
+}
+
+trait ElementBaseView extends ElementDeclView with TermView {
+  def name: String
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    if (isComplexType) {
+      walker.onTypeBegin(Right(complexType))
+      complexType.group.walkDSOM(walker)
+      walker.onTypeBegin(Right(complexType))
+    } else {
+      walker.onTypeBegin(Left(simpleType))
+      walker.onTypeBegin(Left(simpleType))
+    }
+    walker.onTermEnd(this)
+  }
+
+}
+
+trait RootView extends ElementBaseView
+
+/**
+ * A class designed to walk the internal representation of a DFDL Schema File.
+ *
+ * There are 2 main event handlers an implementing class has to worry about: one for Terms, and
+ * another for Types.  These are called as the DSOM is walked.
+ *
+ * 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.
+ * @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] {
+
+  /**
+   * 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: RootView): 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: RootView): T
+
+  /**
+   * Method to be called whenever any element that is a Term is encountered.
+   * This applies to Sequence, Choice, GroupRef, ElementBase, etc.
+   *
+   * It is highly recommended that, when implementing this method, you pattern match
+   * some of these different sub-types at some point to handle each accordingly.
+   *
+   * @param termElement the term element
+   */
+  def onTermBegin(termElement: TermView): Unit
+
+  /**
+   * Method to be called when a Term element has finished processing
+   *
+   * @param termElement the term element
+   */
+  def onTermEnd(termElement: TermView): Unit
+
+  /**
+   * Method to be called when either a Simple or Complex Type element has been encountered.
+   * This is just the element for <simpleType> or <complexType>, so no implementation is necessary if you
+   * do not care about these wrapper portion; its children will be called automatically.
+   *
+   * @param typeElement either a Simple or Complex type
+   */
+  def onTypeBegin(typeElement: Either[SimpleTypeView, ComplexTypeView]): Unit

Review comment:
       Either's are a scala construct, which means this would likely be difficult, or at least uncomfortable, for a Java user to use. Perhaps we have a shared parent of SimpleTypeView and ComplexTypeView and require the user to use isntanceOf, similar to the onTerm methods?

##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,163 @@
+/*
+ * 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 org.apache.daffodil.dpath.NodeInfo.PrimType
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+  def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    walker.onTermEnd(this)
+  }
+}
+
+trait SimpleTypeView {
+  def primType: PrimType
+}
+trait ComplexTypeView {
+  def group: ModelGroupView
+}
+
+trait ModelGroupView extends TermView {
+  def groupMembers: Seq[TermView]
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    this.groupMembers.foreach(_.walkDSOM(walker))
+    walker.onTermEnd(this)
+  }
+}
+
+trait ChoiceView extends ModelGroupView
+trait SequenceView extends ModelGroupView
+trait GroupRefView extends ModelGroupView {
+  def isHidden: Boolean
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = if (!isHidden) super.walkDSOM(walker)
+}
+
+trait ElementDeclView extends CommonContextView {
+  def isSimpleType: Boolean
+  def isComplexType: Boolean
+  def simpleType: SimpleTypeView
+  def complexType: ComplexTypeView
+  def hasDefaultValue: Boolean
+}
+
+trait ElementBaseView extends ElementDeclView with TermView {
+  def name: String
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    if (isComplexType) {
+      walker.onTypeBegin(Right(complexType))
+      complexType.group.walkDSOM(walker)
+      walker.onTypeBegin(Right(complexType))
+    } else {
+      walker.onTypeBegin(Left(simpleType))
+      walker.onTypeBegin(Left(simpleType))
+    }
+    walker.onTermEnd(this)
+  }
+
+}
+
+trait RootView extends ElementBaseView
+
+/**
+ * A class designed to walk the internal representation of a DFDL Schema File.
+ *
+ * There are 2 main event handlers an implementing class has to worry about: one for Terms, and
+ * another for Types.  These are called as the DSOM is walked.
+ *
+ * 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.
+ * @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] {

Review comment:
       I looks like the type parameter is only used for onWalkEnd. My conern is that this type parameter might not translate to a Java user very well. I'm not sure how Scala converts these to byte code. Perhaps it just becomes a Java generic and works fine?
   
   But considering it's only used for the onWalkEnd method, and thuse onWalkFromRoot, I'm wondering if adds much. If a implementation of AbstractDSOMWalker needs a return value of onWalkEnd, they can implement onWalkEnd to store state and access that state after onWalkFromRoot returns? It's maybe a little uglier, but maybe avoids type parameter issues with a Java user of this API?

##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,163 @@
+/*
+ * 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 org.apache.daffodil.dpath.NodeInfo.PrimType
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+  def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    walker.onTermEnd(this)
+  }
+}
+
+trait SimpleTypeView {
+  def primType: PrimType
+}
+trait ComplexTypeView {
+  def group: ModelGroupView
+}
+
+trait ModelGroupView extends TermView {
+  def groupMembers: Seq[TermView]
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    this.groupMembers.foreach(_.walkDSOM(walker))
+    walker.onTermEnd(this)
+  }
+}
+
+trait ChoiceView extends ModelGroupView
+trait SequenceView extends ModelGroupView
+trait GroupRefView extends ModelGroupView {
+  def isHidden: Boolean
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = if (!isHidden) super.walkDSOM(walker)
+}
+
+trait ElementDeclView extends CommonContextView {
+  def isSimpleType: Boolean
+  def isComplexType: Boolean
+  def simpleType: SimpleTypeView
+  def complexType: ComplexTypeView
+  def hasDefaultValue: Boolean
+}
+
+trait ElementBaseView extends ElementDeclView with TermView {
+  def name: String
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    if (isComplexType) {
+      walker.onTypeBegin(Right(complexType))
+      complexType.group.walkDSOM(walker)
+      walker.onTypeBegin(Right(complexType))
+    } else {
+      walker.onTypeBegin(Left(simpleType))
+      walker.onTypeBegin(Left(simpleType))
+    }
+    walker.onTermEnd(this)
+  }
+
+}
+
+trait RootView extends ElementBaseView
+
+/**
+ * A class designed to walk the internal representation of a DFDL Schema File.
+ *
+ * There are 2 main event handlers an implementing class has to worry about: one for Terms, and
+ * another for Types.  These are called as the DSOM is walked.
+ *
+ * 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.
+ * @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] {
+
+  /**
+   * 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: RootView): 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: RootView): T
+
+  /**
+   * Method to be called whenever any element that is a Term is encountered.
+   * This applies to Sequence, Choice, GroupRef, ElementBase, etc.
+   *
+   * It is highly recommended that, when implementing this method, you pattern match
+   * some of these different sub-types at some point to handle each accordingly.
+   *
+   * @param termElement the term element
+   */
+  def onTermBegin(termElement: TermView): Unit

Review comment:
       One concern with Term is that it potentially requires the user to pattern match to figure out what the term is and what to do with it. This is very natural in Scala, but in Java I think this is going to be a much of if-statements with various ``instanceOf``'s and type casting. Maybe not a big deal? I'm not sure.

##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/Term.scala
##########
@@ -18,16 +18,19 @@
 package org.apache.daffodil.dsom
 
 import java.util.UUID
+
 import org.apache.daffodil.exceptions.Assert
 import org.apache.daffodil.grammar.TermGrammarMixin
 import org.apache.daffodil.schema.annotation.props.gen.YesNo
-import java.lang.{ Integer => JInt }
+import java.lang.{Integer => JInt}

Review comment:
       The style we standardized on is to use spaces, though I think that's not the default in Eclipse. There should be a setting to change this though.

##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,163 @@
+/*
+ * 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 org.apache.daffodil.dpath.NodeInfo.PrimType
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+  def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {

Review comment:
       Do we want these walkDSOM function to be protected? Is there a reason that user might want to call walkDSOM directrly? We can also make them public in the future if needed, but it's much harder to switch to make them protected if they are already public.

##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/SimpleTypes.scala
##########
@@ -53,9 +54,9 @@ trait TypeBase {
 trait NonPrimTypeMixin
 
 sealed trait SimpleTypeBase extends TypeBase
-  with HasOptRepTypeMixin {
+  with HasOptRepTypeMixin with SimpleTypeView {
 
-  def primType: PrimType
+  override def primType: PrimType

Review comment:
       Giving direct access to PrimType is potentially problematic from a backwards compatibility perspective. This is something that has changed fairly frequently. I wonder if we also want a walker specific view of PrimType's as well?

##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/compiler/Compiler.scala
##########
@@ -99,6 +99,8 @@ final class ProcessorFactory private(
       new SchemaSet(optRootSpec, schemaSource, validateDFDLSchemas, checkAllTopLevel, tunables,
         compilerExternalVarSettings))
 
+  def walkDSOM[T](walker: AbstractDSOMWalker[T]): T = walker.walkFromRoot(sset.root)

Review comment:
       Normally one would call ``processorFactory.onPath(..)``, which asserts that there are no errors and I think can result in finishing some lazy compilation? Two questions, more directed at Mike who I think knows more about the internal compialtion steps than me:
   1. Should we also assert here that there are no errors?
   2. Do we need to also do something to account for lazy compilation, such as ExecutionMode.usingCompilerMode? Is it possible for an SDE to get hit while walking and performing lazy compliation, or should all that be finished by the time onPath is called?

##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/ModelGroup.scala
##########
@@ -177,7 +180,7 @@ abstract class ModelGroup(index: Int)
    *
    * This also depends on groupMembersDef overrides all being lazy val.
    */
-  final lazy val groupMembers: Seq[Term] =
+  override final lazy val groupMembers: Seq[Term] =

Review comment:
       Giving access to groupMembers probably makes it so the whole walker stuff isn't really necessary. Someone could walk the DSOM entirely with their own implementation rather than relying on all the callbacks and having to store state. Not a big deal I guess, and maybe a benefit?




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455462202



##########
File path: daffodil-core/src/test/scala/org/apache/daffodil/dsom/walker/TestWalker.scala
##########
@@ -0,0 +1,335 @@
+/*
+ * 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 scala.collection.mutable.ListBuffer
+
+/**
+ * This represents all possible types a Field could be; the View Mixin is pattern matched to one
+ * of these in TestNode
+ */
+object FieldType extends Enumeration {
+  type FieldType = Value
+  val RecordType, ChoiceType, ArrayType, DateTimeType, DateType, TimeType, BooleanType, ByteType,
+  ShortType, IntType, LongType, BigIntType, FloatType, DoubleType, StringType = Value
+}
+
+import org.apache.daffodil.dsom.walker.FieldType._
+
+/**
+ * Represents the DataType of a TestField; note that all of the Simple types manifest as
+ * a SimpleDataType with a single FieldType containing the FieldType enum for the simple.
+ * @param fieldType the particular FieldType Enum that this DataType backs.
+ */
+class DataType(val fieldType: FieldType.Value)
+case class RecordDataType(var childSchema: TestSchema) extends DataType(RecordType)
+case class ChoiceDataType(var possibleSubTypes: List[DataType]) extends DataType(ChoiceType)
+case class ArrayDataType(var elementType: DataType) extends DataType(ArrayType)
+case class SimpleDataType(simpleType: FieldType) extends DataType(simpleType)
+
+/**
+ * A field within a TestSchema
+ * @param fieldName name of the field
+ * @param dataType dataType of the field
+ */
+class TestField(val fieldName: String, val dataType: DataType)
+
+/**
+ * Base Schema object for this test class.  The only part that is really used is the field list;
+ * there are some helper methods based around the list
+ * @param name the name for this TestSchema
+ * @param namespace the target namespace of this TestSchema
+ * @param fields the list of fields that define this TestSchema
+ */
+class TestSchema(val name: String, val namespace: String, val fields: List[TestField]) {
+  def getField(name: String): Option[TestField] =
+    fields.filter(field => field.fieldName == name) match {
+      case List(result) => Some(result)
+      case _ => None
+    }
+  val fieldCount: Int = fields.size
+  val fieldNames: List[String] = fields.map(field => field.fieldName)
+}
+
+
+/**
+ * @param fieldName the name for the RecordField
+ * @param dataType the DataType of this RecordField
+ */
+class OptionalTestField(fieldName: String, dataType: DataType)
+  extends TestField(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: TestField) = this(recordField.fieldName, recordField.dataType)
+}
+
+/**
+ * Concrete implementation of the AbstractDSOMWalker abstract class.
+ * This class produces a TestSchema that is intended to match the original DFDL file.
+ *
+ * The TestSchema is built in 3 primary stages:
+ * 1) A tree of TestNodes 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 TestNodes undergoes some post-processing, mainly to remove redundant Record wrappers.
+ * 3) The tree of TestNodes is converted into a TestSchema; it is walked recursively within this class.
+ */
+class TestWalker extends AbstractDSOMWalker {
+
+  // this is the critical data structure for managing the temporary TestNodes that are created
+  // when the Schema is initially walked.  This will then be converted to the actual TestSchema.
+  private var objectStack: List[TestNode] = List()
+
+  // this will contain the final TestSchema after step 3 as described above is complete.
+  private var result: Option[TestSchema] = None
+  def getResult: TestSchema = result.orNull
+  def stringRep: String = if (result.isDefined) result.get.toString else ""
+
+  /**
+   * Every event handler for onXBegin is routed to this helper method.  Individual properties of
+   * a given element are not actually handled here; these properties are extracted when a
+   * TestNode is instantiated
+   * @param dsomElement the particular element that was encountered.  Could be essentially any Term element.
+   */
+  private def onNodeBegin(dsomElement: TermView): Unit = {
+    val newNode: TestNode = new TestNode(dsomElement)
+    // we need add the new node as a new child of whatever is currently at the top of the stack
+    objectStack.head.addChild(newNode)
+    // however, we also add the node to the stack itself!  We need to be able to add children to it
+    // if it is, say, another record or array.
+    objectStack = newNode +: objectStack
+  }
+
+  /**
+   * Every event handler for onXEnd is routed to this helper method.  We don't need to do anything other
+   * than remove the top element of the stack; this should be the current element.
+   */
+  private def onNodeEnd(): Unit = objectStack = objectStack.tail
+
+  override def onTermBegin(termElement: TermView): Unit = termElement match {
+    case _: SequenceView | _: ChoiceView | _: ElementBaseView => onNodeBegin(termElement)
+    case _ =>
+  }
+
+  override def onTermEnd(termElement: TermView): Unit = termElement match {
+    case _: SequenceView | _: ChoiceView | _: ElementBaseView => onNodeEnd()
+    case _ =>
+  }
+
+  override def onTypeBegin(typeElement: TypeView): Unit = {}
+
+  override def onTypeEnd(typeElement: TypeView): Unit = {}
+
+  override def onWalkBegin(root: RootView): Unit = objectStack = List(new TestNode(root))
+
+  override def onWalkEnd(root: RootView): Unit = {
+    // After the walk is over, we perform postProcessing and then convert the TestNode tree
+    // into a TestSchema.  Also, if we are in dev. mode, we print out the TestNode tree
+    if (!RecordWalker.PRODUCTION_MODE) println(objectStack.head)
+    postProcessing()
+    if (!RecordWalker.PRODUCTION_MODE) println(objectStack.head)
+    result = Some(getRecordSchema(objectStack.head))
+  }
+
+  /**
+   * Determines if a Record is "extra"; that is, if it should be replaced with
+   * its list of children SchemaNodes within whatever parent TestNode it's a part of
+   * @param childNode the node to be considered
+   * @return true if this node is a Record and has no name, false otherwise
+   */
+  private def isExtraRecord(parentNode: TestNode, childNode: TestNode): Boolean = {
+    // any no-name nodes with no children are immediately removed
+    (childNode.name.isEmpty && childNode.children.isEmpty) || {
+      parentNode.fieldType match {
+        case RecordType | ArrayType =>
+          childNode.fieldType match {
+            // This removes extra wrapper records around children of records or arrays
+            // usually used to remove things like the DFDL complexType, simpleType elements
+            case RecordType => childNode.name.isEmpty
+            case _ => false
+          }
+        // Currently, all double choices are removed.  This was mainly done to make GroupRefs work
+        // for the JPEG Schema, but may not be the correct approach for all cases.
+        case ChoiceType =>
+          childNode.fieldType match {
+            case ChoiceType => true
+            case _ => false
+          }
+        case _ => false
+      }
+    }
+
+  }
+
+  /**
+   * Recursively replace any SchemaNodes that are of type record and do not have
+   * a name attribute with their children. These usually represent unnecessary wrapper nodes or empty
+   * records with no elements.
+   *
+   * Given a TestNode, we copy over all the children to a list, and initialize another empty list.
+   * Then, for every Node in the first list, if it is considered "extra",
+   * then its *children* are added to the second list.
+   * Otherwise, the Node itself is added.
+   *
+   * Then, we let the first list equal the second and we repeat this process
+   * until none of the children are "extra".
+   * This is necessary because sometimes we can have towers of extra Nodes that would
+   * never get resolved if we just took care of 1 or 2 layers; all must be dealt with at once.
+   *
+   * Then, the current TestNode's children list is replaced with this filtered one,
+   * and this method is called on all the items of that list.
+   * @param schemaNode the current Node undergoing the algorithm described above
+   */
+  private def removeExtraRecords(schemaNode: TestNode): Unit = {
+    schemaNode.fieldType match {
+      // Only SchemaNodes that are these 3 types should be able to have child nodes.
+      case RecordType | ChoiceType | ArrayType =>
+        if (schemaNode.children.exists(child => isExtraRecord(schemaNode, child))) {
+          var currChildren: ListBuffer[TestNode] = new ListBuffer[TestNode]()
+          for (child <- schemaNode.children) currChildren += child
+          while (currChildren.exists(child => isExtraRecord(schemaNode, child))) {
+            val toCopy: ListBuffer[TestNode] = new ListBuffer[TestNode]()
+            for (child <- currChildren) {
+              if (isExtraRecord(schemaNode, child)) {
+                for (subChild <- child.children) toCopy += subChild
+              } else toCopy += child
+            }
+            currChildren = toCopy
+          }
+          schemaNode.children = currChildren.toList
+        }
+      case _ =>
+    }
+    // call this helper method on each of this Nodes's children.  There is no infinite recursion because
+    // eventually a sub-child will not have any more children.  This of course assumes that we are dealing
+    // with a rooted tree (there shouldn't be any cycles), which should be the case.
+    for (child <- schemaNode.children) removeExtraRecords(child)
+  }
+
+  /**
+   * Perform postProcessing; this happens *after* the TestNode tree is created but *before* that tree
+   * gets converted to a TestSchema
+   */
+  private def postProcessing(): Unit = {
+    removeExtraRecords(objectStack.head)
+  }
+
+  private def getSimpleType(schemaNode: TestNode): FieldType = {
+    if (!schemaNode.isSimple) throw new IllegalArgumentException(s"Node $schemaNode is not a simple type!")
+    schemaNode.fieldType
+  }
+
+  private def nodeToField(schemaNode: TestNode): TestField = {
+    // by default, if this node doesn't have a name, its data type is used as the field name.
+    // This should only ever be the case for anonymous choices.
+    val recordField: TestField = new TestField(
+      schemaNode.name.getOrElse(schemaNode.fieldType.toString),
+      schemaNodeToDataType(schemaNode)
+    )
+    if (schemaNode.isOptional) new OptionalTestField(recordField) else recordField
+  }
+
+  private def newRecordSchema(schemaNode: TestNode, fieldList: List[TestField]): TestSchema = {
+    new TestSchema(schemaNode.name.getOrElse(""), schemaNode.namespace, fieldList)
+  }
+
+  /**
+   * Helper method to specifically convert a TestNode known to be a Record into a TestSchema.
+   * Each Child of the TestNode becomes a Field of the TestSchema
+   * @param schemaNode the node to convert to a TestSchema
+   * @return a tuple containing the finalized TestSchema and its corresponding Record Data Type
+   */
+  private def schemaNodeToRecordType(schemaNode: TestNode): RecordDataType = {
+    val fieldList: ListBuffer[TestField] = ListBuffer()
+    for (child <- schemaNode.children) fieldList += nodeToField(child)
+    val TestSchema: TestSchema = newRecordSchema(schemaNode, fieldList.toList)
+    RecordDataType(TestSchema)
+  }
+
+  /**
+   * Helper method to convert a TestNode known to be a choice into a Choice data type.
+   * This is able to handle a DFDL schema in which either named elements are directly sub-members of
+   * DFDL choices or if they are embedded in another element (which corresponds to being in a Record).
+   * In the end, if they are not already inside a Record, then they are put there. Choices cannot
+   * have fields, only possible sub-types, so anything that would be a "field" has a wrapper Record
+   * put around it.
+   * @param schemaNode the node to convert to a Choice data type
+   * @return a Choice data type as described above
+   */
+  private def choiceNodeToChoiceSchema(schemaNode: TestNode): DataType = {
+    val childList: ListBuffer[DataType] = new ListBuffer()
+    for (child <- schemaNode.children) {
+      if (child.name.isEmpty) {
+        childList += schemaNodeToRecordType(child)
+      } else {
+        // In the case where the child has a name, we embed it in an additional TestSchema,
+        // since we must add some DataType rather than a Field.
+        val fieldList: ListBuffer[TestField] = ListBuffer()
+        fieldList += nodeToField(child)
+        val choiceOption: TestSchema = newRecordSchema(child, fieldList.toList)
+        childList += RecordDataType(choiceOption)
+      }
+    }
+    ChoiceDataType(childList.toList)
+  }
+
+  /**
+   * Local helper method to appropriately convert a TestNode into an appropriate Record
+   * Data Type.  Records and Choices get routed to other helper methods, and Arrays are handled in the method.
+   * @param schemaNode the node to convert to a Record Data Type
+   * @return the finalized Record Data type
+   */
+  private def schemaNodeToDataType(schemaNode: TestNode): DataType = {
+    schemaNode.fieldType match {
+      case ArrayType =>
+        if (schemaNode.isSimple) {
+          ArrayDataType(new DataType(getSimpleType(schemaNode)))
+        } else if (schemaNode.children.nonEmpty) {
+          ArrayDataType(schemaNodeToRecordType(schemaNode))
+        } else {
+          throw new IllegalStateException(
+            s"Array Node $schemaNode either needs to be simple or have child nodes!"
+          )
+        }
+      case RecordType => schemaNodeToRecordType(schemaNode)
+      case ChoiceType => choiceNodeToChoiceSchema(schemaNode)
+      case _ => schemaNode.simpleType.get
+    }
+  }
+
+  /**
+   * In general, this method can be used to extract the TestSchema node from the tuple result
+   * of schemaNodeToRecordSchema.  However, it is really only used to kick off the conversion from
+   * the root TestNode.
+   * @param schemaNode the node from which a TestSchema will be extracted
+   * @return the processed TestSchema represented by the input
+   */
+  private def getRecordSchema(schemaNode: TestNode): TestSchema = {
+    schemaNodeToRecordType(schemaNode).childSchema
+  }
+
+}
+
+/**
+ * The sole purpose of this Companion Object is to contain this one static variable.
+ */
+object RecordWalker {
+  /**
+   * Used to suppress all of the extra printing from running test cases
+   */
+  final val PRODUCTION_MODE: Boolean = true

Review comment:
       I did not wind up ignoring things like complexType and simpleType in the testComplexTypes() test case, but I do have a flag for that in the new testing class if that is desired - let me know if you prefer that.




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r456009313



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,174 @@
+/*
+ * 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
+
+/**
+ * Below is a series of "View" Mixins defined such that particular simple attributes
+ * of the core DSOM classes are exposed as a Walk is performed.
+ *
+ * Several of them have a walkDSOM method.  By default, this method is called as the walk is performed,
+ * but a user implementing AbstractDSOMWalker can override this behavior by overriding walkFromRoot
+ * and implementing their own event handlers for each of the DSOM types.
+ */
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+}
+
+trait TypeView
+trait SimpleTypeView extends TypeView {
+  def primType: PrimTypeView
+}
+trait ComplexTypeView extends TypeView {
+  def group: ModelGroupView
+}
+
+trait ModelGroupView extends TermView {
+  def groupMembers: Seq[TermView]
+}
+
+trait ChoiceView extends ModelGroupView
+trait SequenceView extends ModelGroupView
+trait GroupRefView extends ModelGroupView {
+  def isHidden: Boolean

Review comment:
       I think there is a solution to this, but it would break the current paradigm of every field in the `View`s referencing an existing field.  We could replace the existing `group` element in `ComplexTypeView` with something like `optVisibleGroup`, and then the implementation for `optVisibleGroup` would pattern-match `group`; if it is hidden, return `None, otherwise return the `group` wrapped in an option.  Similarly, instead of `groupMembers` being in `TermView`, we would have `visisbleGroupMembers`, and in the implementation `visibleGroupMembers` would be a filtered list of the real `groupMembers`, excluding any `TermView` that is a `GroupRef` and is also hidden.
   
   Now, my main concern with this is it would potentially cause more confusion, leading the end user to wonder why we're only allowed to see "visible" members, and might make traversal very annoying with having to constantly unpack Options.




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455887835



##########
File path: daffodil-core/src/test/scala/org/apache/daffodil/dsom/walker/TestDSOMWalker.scala
##########
@@ -0,0 +1,229 @@
+/*
+ * 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 org.apache.daffodil.util._
+import org.apache.daffodil.compiler.{ Compiler, ProcessorFactory }
+import org.junit.Test
+import org.junit.Assert._
+
+class TestDSOMWalker {
+
+  @Test def testComplexTypes(): Unit = {
+    val testSchema = SchemaUtils.dfdlTestSchema(
+      <xs:include schemaLocation="org/apache/daffodil/xsd/DFDLGeneralFormat.dfdl.xsd"/>,
+      <dfdl:format ref="ex:GeneralFormat"
+                   alignment="implicit" alignmentUnits="bits" occursCountKind="implicit"
+                   lengthKind="delimited" encoding="ASCII"/>,
+      <xs:element name="PersonData">
+        <xs:complexType>
+          <xs:sequence>
+            <xs:choice>
+              <xs:element name="age" type="xs:int" minOccurs="1" maxOccurs="1"/>
+            </xs:choice>
+            <xs:group ref="testGroup" />
+          </xs:sequence>
+        </xs:complexType>
+      </xs:element>
+      <xs:group name="testGroup">
+        <xs:sequence />
+      </xs:group>
+    )
+    val pf: ProcessorFactory = Compiler().compileNode(testSchema)
+    assertEquals(s"This basic Schema: $testSchema should compile; here are some diagnostics: ${pf.getDiagnostics}", false, pf.isError)
+    val walker: BasicWalker = new BasicWalker()
+    walker.walkFromRoot(pf.rootView)
+    val nodeStack: List[Either[TermView, TypeView]] = walker.nodeArr.toList
+    assertEquals(s"Node Stack $nodeStack did not have the expected number of elements", 7, nodeStack.size)
+    nodeStack.head match {
+      case Left(personElement: RootView) =>
+        assertEquals("The root element should be named 'PersonData'", "PersonData", personElement.name)
+      case Left(someClass) => fail(s"The Root element was not of type RootView, it was a ${someClass.getClass}!")
+      case Right(someClass) => fail(s"The Root element was not of type RootView, it was a ${someClass.getClass}!")
+    }
+    nodeStack(1) match {
+      case Right(_: ComplexTypeView) =>
+      case Left(someClass) => fail(s"The Root element did not contain a complexType wrapper child, it was a ${someClass.getClass}!")
+      case Right(someClass) => fail(s"The Root element element did not contain a complexType wrapper child, it was a ${someClass.getClass}!")
+    }
+    nodeStack(2) match {
+      case Left(_: SequenceView) =>
+      case Left(someClass) => fail(s"The complexType element did not contain a Sequence child, it was a ${someClass.getClass}!")
+      case Right(someClass) => fail(s"The complexType element did not contain a Sequence child, it was a ${someClass.getClass}!")
+    }
+    nodeStack(3) match {
+      case Left(_: ChoiceView) =>
+      case Left(someClass) => fail(s"The Sequence element did not contain a Choice child, it was a ${someClass.getClass}!")
+      case Right(someClass) => fail(s"The Sequence element did not contain a Choice child, it was a ${someClass.getClass}!")
+    }
+    nodeStack(4) match {
+      case Left(nameElement: ElementBaseView) =>
+        assertEquals("The second child element should be named 'age'", "age", nameElement.name)
+        assertEquals("The second child element should be a simple type", true, nameElement.isSimpleType)
+      case Left(someClass) => fail(s"The Sequence element did not contain a second Element child, it was a ${someClass.getClass}!")
+      case Right(someClass) => fail(s"The Sequence element did not contain a second Element child, it was a ${someClass.getClass}!")
+    }
+    nodeStack(5) match {
+      case Right(simpleType: SimpleTypeView) =>
+        simpleType.primType match {
+          case _:IntView =>
+          case _ => fail(s"The 'age' element should be of simple type Int")
+        }
+      case Left(someClass) => fail(s"The 'age' element did not have a SimpleTypeView, it was a ${someClass.getClass}!")
+      case Right(someClass) => fail(s"The 'age' element did not have a SimpleTypeView, it was a ${someClass.getClass}!")
+    }
+    nodeStack(6) match {
+      case Left(_: GroupRefView) =>
+      case Left(someClass) => fail(s"The Sequence element did not contain a GroupRef child, it was a ${someClass.getClass}!")
+      case Right(someClass) => fail(s"The Sequence element did not contain a GroupRef child, it was a ${someClass.getClass}!")
+    }
+  }

Review comment:
       I agree that it is robust, and I had it this way mainly for the sake of pattern matching and being able to figure out types of specific View elements on the stack when tests were failing.  That being said, for simplicity sake, I think the series of asserts based on `isInstanceOf` and `asInstanceOf` would be best.  I tried implementing a case class with an equals method that takes in an `AnyRef` as an argument as you suggested above, and it turned out to be an even longer solution.




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455968808



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,177 @@
+/*
+ * 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
+
+/**
+ * Below is a series of "View" Mixins defined such that particular simple attributes
+ * of the core DSOM classes are exposed as a Walk is performed.
+ *
+ * Several of them have a walkDSOM method.  By default, this method is called as the walk is performed,
+ * but a user implementing AbstractDSOMWalker can override this behavior by overriding walkFromRoot
+ * and implementing their own event handlers for each of the DSOM types.
+ */
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+}
+
+trait TypeView
+trait SimpleTypeView extends TypeView {
+  def primType: PrimTypeView
+}
+trait ComplexTypeView extends TypeView {
+  def group: ModelGroupView
+}
+
+trait ModelGroupView extends TermView {
+  def groupMembers: Seq[TermView]
+}
+
+trait ChoiceView extends ModelGroupView
+trait SequenceView extends ModelGroupView
+trait GroupRefView extends ModelGroupView {
+  def isHidden: Boolean
+}
+
+trait ElementDeclView extends CommonContextView {
+  def isSimpleType: Boolean
+  def isComplexType: Boolean
+  def simpleType: SimpleTypeView
+  def complexType: ComplexTypeView
+  def hasDefaultValue: Boolean
+}
+
+trait ElementBaseView extends ElementDeclView with TermView {
+  def name: String
+}
+
+trait RootView extends ElementBaseView
+
+/**
+ * A class designed to walk the internal representation of a DFDL Schema File.
+ *
+ * There are 2 main event handlers an implementing class has to worry about: one for Terms, and
+ * another for Types.  These are called as the DSOM is walked.
+ *
+ * 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.
+ */
+abstract class AbstractDSOMWalker {
+
+  /**
+   * 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: RootView): 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
+   */
+  protected def onWalkEnd(root: RootView): Unit
+
+  /**
+   * Method to be called whenever any element that is a Term is encountered.
+   * This applies to Sequence, Choice, GroupRef, ElementBase, etc.
+   *
+   * It is highly recommended that, when implementing this method, you pattern match
+   * some of these different sub-types at some point to handle each accordingly.
+   *
+   * @param termElement the term element
+   */
+  def onTermBegin(termElement: TermView): Unit
+
+  /**
+   * Method to be called when a Term element has finished processing
+   *
+   * @param termElement the term element
+   */
+  def onTermEnd(termElement: TermView): Unit
+
+  /**
+   * Method to be called when either a Simple or Complex Type element has been encountered.
+   * This is just the element for <simpleType> or <complexType>, so no implementation is necessary if you
+   * do not care about these wrapper portion; its children will be walked automatically.
+   *
+   * @param typeElement either a Simple or Complex type
+   */
+  def onTypeBegin(typeElement: TypeView): Unit
+
+  /**
+   * Method to be called when either a Simple or Complex Type element has been finished
+   * processing.  See onTypeBegin method description
+   *
+   * @param typeElement either a Simple or Complex type
+   */
+  def onTypeEnd(typeElement: TypeView): Unit
+
+  /**
+   * Starts a DSOM walk from the Root object.  By default, the walk is completed recursively in the helper
+   * method below, but a custom walk can be defined by overriding
+   * this method instead and calling local event handlers.
+   * @param schemaSetRoot The root element of the DFDL Schema.  This will be the starting point of the traversal
+   */
+  def walkFromRoot(schemaSetRoot: RootView): Unit = {
+    onWalkBegin(schemaSetRoot)
+    walkerHelper(schemaSetRoot)
+    // onWalkEnd will return the value returned by this method!
+    onWalkEnd(schemaSetRoot)
+  }
+
+  /**
+   * Local helper method used to recursively walk the DSOM.
+   * If a non-recursive walk or a more specific recursive walk is desired, one
+   * can override this method, or simply not use it and instead override walkFromRoot
+   * and have that overridden method reference custom helper methods.
+   * @param termView the current Term element to be pattern matched
+   */
+  protected def walkerHelper(termView: TermView): Unit = {
+    onTermBegin(termView)
+    termView match {
+      case element: ElementBaseView =>
+        if (element.isComplexType) {
+          onTypeBegin(element.complexType)
+          walkerHelper(element.complexType.group)
+          onTypeEnd(element.complexType)
+        } else {
+          onTypeBegin(element.simpleType)
+          onTypeEnd(element.simpleType)
+        }
+      case groupRef: GroupRefView =>
+        if (!groupRef.isHidden) {
+          groupRef.groupMembers.foreach(walkerHelper)
+        }
+      case modelGroup: ModelGroupView =>
+        modelGroup.groupMembers.foreach(walkerHelper)
+      case _ =>

Review comment:
       As the View structure currently stands, the default case will not ever get hit; removed this case clause in the latest revision.




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453128358



##########
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:
       While I did leave the fundamental recursive walk structure intact, as Mike stated might have to do anyway in the end, I did consolidate quite a few methods.  Now the user need only implement events for starting and ending the walk, and 2 handlers each for Terms and Types, where a Type could either be Simple or Complex.  See the latest commit.




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



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

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453842979



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/Term.scala
##########
@@ -18,16 +18,19 @@
 package org.apache.daffodil.dsom
 
 import java.util.UUID
+
 import org.apache.daffodil.exceptions.Assert
 import org.apache.daffodil.grammar.TermGrammarMixin
 import org.apache.daffodil.schema.annotation.props.gen.YesNo
-import java.lang.{ Integer => JInt }
+import java.lang.{Integer => JInt}

Review comment:
       I don't use IntelliJ, but here's our page on IntelliJ setup, Mike added some comments about some style changes:
   
   https://cwiki.apache.org/confluence/display/DAFFODIL/IntelliJ+IDEA+Setup




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



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

Posted by GitBox <gi...@apache.org>.
tuxji commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r452926725



##########
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:
       I like Steve's suggestion to separate compilation and walking, and provide a DSOM walk method which takes a Listener class (if I understand correctly).  I have worked on someone else's Visitor class for an ANTLR v4 grammar and then written my own Listener class for the same ANTLR v4 grammar.  In my experience, the Listener class for the same use case was smaller and more maintainable since ANTLR took care of visiting every part of the parse tree (smaller chance for mistakes) and I was able to implement only a handful of listener methods that I actually needed.  ANTLR listeners can't return values directly from method calls, but I was able to attach these return values to specific parts of the parse tree (by modifying the ANTLR grammar to provide fields for these values) and have higher up listener methods collect these values from lower down parts of the parse tree.  Collecting return values may become harder if you can't store them in the tree as you walk the tree like I did, though.
   
   In case people aren't familiar with ANTLR visitors and walkers, someone has blogged about them (<https://saumitra.me/blog/antlr4-visitor-vs-listener-pattern/>) and said:
   
   > Good thing about listener pattern is that its all automatic and even if you don’t write any parse tree walker, ANTLR will figure out and trigger the enter and exit method for each rule. This is a huge benefit for translation type of usecases.




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453129119



##########
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:
       This all makes a lot of sense, hopefully I took care of it well in the latest version.
   
   I wound up defining a series of traits ending in View as Mike suggested, such as TermView and SequenceView.  Everything in RecordWalker and the helper SchemaNode (which I renamed to RecordSchemaNode to avoid confusion, it should _only_ be used for the NIFI RecordSchema, as it has an attribute that is evaluated to a NIFI Record Simple type, now uses all of these traits.
   
   The only "internal" item the traits can access that isn't a primitive or a String is the Enum PrimType class; I decided to keep it since it was highly useful when it came to simple types.  I hope leaving this in won't cause any issues.
   
   Within these traits I also defined a new "walkDSOM" method; hence moving all of those walkerHelper methods out of the abstract class and into the new traits.




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453865556



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,163 @@
+/*
+ * 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 org.apache.daffodil.dpath.NodeInfo.PrimType
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+  def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    walker.onTermEnd(this)
+  }
+}
+
+trait SimpleTypeView {
+  def primType: PrimType
+}
+trait ComplexTypeView {
+  def group: ModelGroupView
+}
+
+trait ModelGroupView extends TermView {
+  def groupMembers: Seq[TermView]
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    this.groupMembers.foreach(_.walkDSOM(walker))
+    walker.onTermEnd(this)
+  }
+}
+
+trait ChoiceView extends ModelGroupView
+trait SequenceView extends ModelGroupView
+trait GroupRefView extends ModelGroupView {
+  def isHidden: Boolean
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = if (!isHidden) super.walkDSOM(walker)
+}
+
+trait ElementDeclView extends CommonContextView {
+  def isSimpleType: Boolean
+  def isComplexType: Boolean
+  def simpleType: SimpleTypeView
+  def complexType: ComplexTypeView
+  def hasDefaultValue: Boolean
+}
+
+trait ElementBaseView extends ElementDeclView with TermView {
+  def name: String
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    if (isComplexType) {
+      walker.onTypeBegin(Right(complexType))
+      complexType.group.walkDSOM(walker)
+      walker.onTypeBegin(Right(complexType))
+    } else {
+      walker.onTypeBegin(Left(simpleType))
+      walker.onTypeBegin(Left(simpleType))
+    }
+    walker.onTermEnd(this)
+  }
+
+}
+
+trait RootView extends ElementBaseView
+
+/**
+ * A class designed to walk the internal representation of a DFDL Schema File.
+ *
+ * There are 2 main event handlers an implementing class has to worry about: one for Terms, and
+ * another for Types.  These are called as the DSOM is walked.
+ *
+ * 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.
+ * @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] {
+
+  /**

Review comment:
       It is possible to build nars using scala code, I previously had all of this code in a maven project that mixed NIFI-specific Java code and Scala code.  It only required a few additional packages to include.
   
   That being said, if the NiFi developer doesn't have to code in Scala, they don't have to worry about any of that - they would only need to do this configuration if they wanted to implement AbstractDSOMWalker in Scala




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453977401



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,163 @@
+/*
+ * 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 org.apache.daffodil.dpath.NodeInfo.PrimType
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+  def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    walker.onTermEnd(this)
+  }
+}
+
+trait SimpleTypeView {
+  def primType: PrimType
+}
+trait ComplexTypeView {
+  def group: ModelGroupView
+}
+
+trait ModelGroupView extends TermView {
+  def groupMembers: Seq[TermView]
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    this.groupMembers.foreach(_.walkDSOM(walker))
+    walker.onTermEnd(this)
+  }
+}
+
+trait ChoiceView extends ModelGroupView
+trait SequenceView extends ModelGroupView
+trait GroupRefView extends ModelGroupView {
+  def isHidden: Boolean
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = if (!isHidden) super.walkDSOM(walker)
+}
+
+trait ElementDeclView extends CommonContextView {
+  def isSimpleType: Boolean
+  def isComplexType: Boolean
+  def simpleType: SimpleTypeView
+  def complexType: ComplexTypeView
+  def hasDefaultValue: Boolean
+}
+
+trait ElementBaseView extends ElementDeclView with TermView {
+  def name: String
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    if (isComplexType) {
+      walker.onTypeBegin(Right(complexType))
+      complexType.group.walkDSOM(walker)
+      walker.onTypeBegin(Right(complexType))
+    } else {
+      walker.onTypeBegin(Left(simpleType))
+      walker.onTypeBegin(Left(simpleType))
+    }
+    walker.onTermEnd(this)
+  }
+
+}
+
+trait RootView extends ElementBaseView
+
+/**
+ * A class designed to walk the internal representation of a DFDL Schema File.
+ *
+ * There are 2 main event handlers an implementing class has to worry about: one for Terms, and
+ * another for Types.  These are called as the DSOM is walked.
+ *
+ * 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.
+ * @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] {
+
+  /**
+   * 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: RootView): 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: RootView): T
+
+  /**
+   * Method to be called whenever any element that is a Term is encountered.
+   * This applies to Sequence, Choice, GroupRef, ElementBase, etc.
+   *
+   * It is highly recommended that, when implementing this method, you pattern match
+   * some of these different sub-types at some point to handle each accordingly.
+   *
+   * @param termElement the term element
+   */
+  def onTermBegin(termElement: TermView): Unit
+
+  /**
+   * Method to be called when a Term element has finished processing
+   *
+   * @param termElement the term element
+   */
+  def onTermEnd(termElement: TermView): Unit
+
+  /**
+   * Method to be called when either a Simple or Complex Type element has been encountered.
+   * This is just the element for <simpleType> or <complexType>, so no implementation is necessary if you
+   * do not care about these wrapper portion; its children will be called automatically.
+   *
+   * @param typeElement either a Simple or Complex type
+   */
+  def onTypeBegin(typeElement: Either[SimpleTypeView, ComplexTypeView]): Unit

Review comment:
       I opted for creating a shared parent in the latest revision - if we decide to stick with that then yes, in a Java implementation they would have to do instanceOf and casting




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



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

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455775157



##########
File path: daffodil-core/src/test/scala/org/apache/daffodil/dsom/walker/BasicWalker.scala
##########
@@ -0,0 +1,50 @@
+/*
+ * 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 scala.collection.mutable.ArrayBuffer
+
+class BasicWalker(ignoreTypeWrappers: Boolean = false, onlyElements: Boolean = false) extends AbstractDSOMWalker {
+
+  var nodeArr: ArrayBuffer[Either[TermView, TypeView]] = ArrayBuffer()
+
+  override protected def onWalkBegin(root: RootView): Unit = {}
+
+  override protected def onWalkEnd(root: RootView): Unit = {}
+
+  override def onTermBegin(termElement: TermView): Unit = {
+    if (onlyElements) {
+      termElement match {
+        case _: ElementBaseView => nodeArr += Left(termElement)
+        case _ =>
+      }
+    } else {
+      nodeArr += Left(termElement)
+    }
+  }

Review comment:
       I think maybe a cleaner or more scala-y way to write this would be something like this:
   ```scala
   termElement match {
     case _: ElementBaseView => nodeArr += Left(termElement)
     case _=> if (!onlyElements) nodeArr += Left(termElement)
   }
   ```




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455885398



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,177 @@
+/*
+ * 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
+
+/**
+ * Below is a series of "View" Mixins defined such that particular simple attributes
+ * of the core DSOM classes are exposed as a Walk is performed.
+ *
+ * Several of them have a walkDSOM method.  By default, this method is called as the walk is performed,
+ * but a user implementing AbstractDSOMWalker can override this behavior by overriding walkFromRoot
+ * and implementing their own event handlers for each of the DSOM types.
+ */
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+}
+
+trait TypeView
+trait SimpleTypeView extends TypeView {
+  def primType: PrimTypeView
+}
+trait ComplexTypeView extends TypeView {
+  def group: ModelGroupView
+}
+
+trait ModelGroupView extends TermView {
+  def groupMembers: Seq[TermView]
+}
+
+trait ChoiceView extends ModelGroupView
+trait SequenceView extends ModelGroupView
+trait GroupRefView extends ModelGroupView {
+  def isHidden: Boolean
+}
+
+trait ElementDeclView extends CommonContextView {
+  def isSimpleType: Boolean
+  def isComplexType: Boolean
+  def simpleType: SimpleTypeView
+  def complexType: ComplexTypeView
+  def hasDefaultValue: Boolean

Review comment:
       NiFi doesn't need this; I merely included it because I was trying to mimic the structure of `ElementDeclMixin`.  I notice that `ElementDeclMixin` has `defaultValueAsString` member, would that be the correct member to add?  If so, then I could add it, but I could also just remove `hasDefaultValue` from the ElementDeclView if this is not an important attribute.




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455902768



##########
File path: daffodil-japi/src/main/scala/org/apache/daffodil/japi/Daffodil.scala
##########
@@ -373,6 +374,12 @@ class ProcessorFactory private[japi] (private var pf: SProcessorFactory)
     new DataProcessor(dp)
   }
 
+  /**
+   * Exposes the RootView object corresponding to this ProcessorFactory.  This can
+   * be used to start a walk using the walkFromRoot method in a DSOM Walker.
+   */
+  def rootView: RootView = pf.rootView

Review comment:
       What is the preferred way to mark an API as experimental in Daffodil?  Is there an Annotation or ScalaDoc tag, or should I simply state it's Experimental in the comment describing `rootView`?




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



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

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453863360



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,163 @@
+/*
+ * 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 org.apache.daffodil.dpath.NodeInfo.PrimType
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+  def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    walker.onTermEnd(this)
+  }
+}
+
+trait SimpleTypeView {
+  def primType: PrimType
+}
+trait ComplexTypeView {
+  def group: ModelGroupView
+}
+
+trait ModelGroupView extends TermView {
+  def groupMembers: Seq[TermView]
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    this.groupMembers.foreach(_.walkDSOM(walker))
+    walker.onTermEnd(this)
+  }
+}
+
+trait ChoiceView extends ModelGroupView
+trait SequenceView extends ModelGroupView
+trait GroupRefView extends ModelGroupView {
+  def isHidden: Boolean
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = if (!isHidden) super.walkDSOM(walker)
+}
+
+trait ElementDeclView extends CommonContextView {
+  def isSimpleType: Boolean
+  def isComplexType: Boolean
+  def simpleType: SimpleTypeView
+  def complexType: ComplexTypeView
+  def hasDefaultValue: Boolean
+}
+
+trait ElementBaseView extends ElementDeclView with TermView {
+  def name: String
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    if (isComplexType) {
+      walker.onTypeBegin(Right(complexType))
+      complexType.group.walkDSOM(walker)
+      walker.onTypeBegin(Right(complexType))
+    } else {
+      walker.onTypeBegin(Left(simpleType))
+      walker.onTypeBegin(Left(simpleType))
+    }
+    walker.onTermEnd(this)
+  }
+
+}
+
+trait RootView extends ElementBaseView
+
+/**
+ * A class designed to walk the internal representation of a DFDL Schema File.
+ *
+ * There are 2 main event handlers an implementing class has to worry about: one for Terms, and
+ * another for Types.  These are called as the DSOM is walked.
+ *
+ * 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.
+ * @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] {
+
+  /**

Review comment:
       One concern with requireing scala is building a nar for a NiFi controller/processer uses a maven archetype template to build the project structure. If the API requires Scala, building such a nar might be problematic, or at least this generated maven project needs to be modified support scala compilation. I assume that's possible, but I'm not sure. Other integrations might have similar issues.
   
   And if the the utltimate goal is to upstream this into NiFi tree, it almost certainly needs to be Java friendly--as far as I know, NiFi doesn't have any Scala code in their source tree.
   
   It we decide to only support scala (which doesn't seem unreasonable), and if there's a potential for changes to support Java, I would at least recomend we say that this API is experiemental and subject to change so that we don't have to worry as much about backwards compatibility and can more easily make such changes.




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r454721372



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/SimpleTypes.scala
##########
@@ -53,9 +54,9 @@ trait TypeBase {
 trait NonPrimTypeMixin
 
 sealed trait SimpleTypeBase extends TypeBase
-  with HasOptRepTypeMixin {
+  with HasOptRepTypeMixin with SimpleTypeView {
 
-  def primType: PrimType
+  override def primType: PrimType

Review comment:
       I went ahead with that route, it seemed a lot more elegant to integrate the View paradigm without adding any fields, just like what had been done for everything else.  I put all the Views related to PrimType in their own file in runtime-1 and most of the primitive enums now implement them




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



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

Posted by GitBox <gi...@apache.org>.
mbeckerle commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r453890762



##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/dsom/walker/AbstractDSOMWalker.scala
##########
@@ -0,0 +1,163 @@
+/*
+ * 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 org.apache.daffodil.dpath.NodeInfo.PrimType
+
+trait CommonContextView {
+  def namespaces: scala.xml.NamespaceBinding
+}
+
+trait TermView extends CommonContextView {
+  def isArray: Boolean
+  def isOptional: Boolean
+  def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    walker.onTermEnd(this)
+  }
+}
+
+trait SimpleTypeView {
+  def primType: PrimType
+}
+trait ComplexTypeView {
+  def group: ModelGroupView
+}
+
+trait ModelGroupView extends TermView {
+  def groupMembers: Seq[TermView]
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    this.groupMembers.foreach(_.walkDSOM(walker))
+    walker.onTermEnd(this)
+  }
+}
+
+trait ChoiceView extends ModelGroupView
+trait SequenceView extends ModelGroupView
+trait GroupRefView extends ModelGroupView {
+  def isHidden: Boolean
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = if (!isHidden) super.walkDSOM(walker)
+}
+
+trait ElementDeclView extends CommonContextView {
+  def isSimpleType: Boolean
+  def isComplexType: Boolean
+  def simpleType: SimpleTypeView
+  def complexType: ComplexTypeView
+  def hasDefaultValue: Boolean
+}
+
+trait ElementBaseView extends ElementDeclView with TermView {
+  def name: String
+  override def walkDSOM[T](walker: AbstractDSOMWalker[T]): Unit = {
+    walker.onTermBegin(this)
+    if (isComplexType) {
+      walker.onTypeBegin(Right(complexType))
+      complexType.group.walkDSOM(walker)
+      walker.onTypeBegin(Right(complexType))
+    } else {
+      walker.onTypeBegin(Left(simpleType))
+      walker.onTypeBegin(Left(simpleType))
+    }
+    walker.onTermEnd(this)
+  }
+
+}
+
+trait RootView extends ElementBaseView
+
+/**
+ * A class designed to walk the internal representation of a DFDL Schema File.
+ *
+ * There are 2 main event handlers an implementing class has to worry about: one for Terms, and
+ * another for Types.  These are called as the DSOM is walked.
+ *
+ * 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.
+ * @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] {
+
+  /**
+   * 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: RootView): 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: RootView): T
+
+  /**
+   * Method to be called whenever any element that is a Term is encountered.
+   * This applies to Sequence, Choice, GroupRef, ElementBase, etc.
+   *
+   * It is highly recommended that, when implementing this method, you pattern match
+   * some of these different sub-types at some point to handle each accordingly.
+   *
+   * @param termElement the term element
+   */
+  def onTermBegin(termElement: TermView): Unit

Review comment:
       To make this java friendly, one would typically use a visiting pattern so that what we do in scala via a match on types becomes calling a method which is overloaded on types of a parameter. I think we'd have to try it to see how it works out.




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



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

Posted by GitBox <gi...@apache.org>.
andrewjc2000 commented on a change in pull request #394:
URL: https://github.com/apache/incubator-daffodil/pull/394#discussion_r455460265



##########
File path: daffodil-core/src/test/scala/org/apache/daffodil/dsom/walker/TestWalker.scala
##########
@@ -0,0 +1,335 @@
+/*
+ * 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 scala.collection.mutable.ListBuffer
+
+/**
+ * This represents all possible types a Field could be; the View Mixin is pattern matched to one
+ * of these in TestNode
+ */
+object FieldType extends Enumeration {
+  type FieldType = Value
+  val RecordType, ChoiceType, ArrayType, DateTimeType, DateType, TimeType, BooleanType, ByteType,
+  ShortType, IntType, LongType, BigIntType, FloatType, DoubleType, StringType = Value
+}
+
+import org.apache.daffodil.dsom.walker.FieldType._
+
+/**
+ * Represents the DataType of a TestField; note that all of the Simple types manifest as
+ * a SimpleDataType with a single FieldType containing the FieldType enum for the simple.
+ * @param fieldType the particular FieldType Enum that this DataType backs.
+ */
+class DataType(val fieldType: FieldType.Value)
+case class RecordDataType(var childSchema: TestSchema) extends DataType(RecordType)
+case class ChoiceDataType(var possibleSubTypes: List[DataType]) extends DataType(ChoiceType)
+case class ArrayDataType(var elementType: DataType) extends DataType(ArrayType)
+case class SimpleDataType(simpleType: FieldType) extends DataType(simpleType)
+
+/**
+ * A field within a TestSchema
+ * @param fieldName name of the field
+ * @param dataType dataType of the field
+ */
+class TestField(val fieldName: String, val dataType: DataType)
+
+/**
+ * Base Schema object for this test class.  The only part that is really used is the field list;
+ * there are some helper methods based around the list
+ * @param name the name for this TestSchema
+ * @param namespace the target namespace of this TestSchema
+ * @param fields the list of fields that define this TestSchema
+ */
+class TestSchema(val name: String, val namespace: String, val fields: List[TestField]) {
+  def getField(name: String): Option[TestField] =
+    fields.filter(field => field.fieldName == name) match {
+      case List(result) => Some(result)
+      case _ => None
+    }
+  val fieldCount: Int = fields.size
+  val fieldNames: List[String] = fields.map(field => field.fieldName)
+}
+
+
+/**
+ * @param fieldName the name for the RecordField
+ * @param dataType the DataType of this RecordField
+ */
+class OptionalTestField(fieldName: String, dataType: DataType)
+  extends TestField(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: TestField) = this(recordField.fieldName, recordField.dataType)
+}
+
+/**
+ * Concrete implementation of the AbstractDSOMWalker abstract class.
+ * This class produces a TestSchema that is intended to match the original DFDL file.
+ *
+ * The TestSchema is built in 3 primary stages:
+ * 1) A tree of TestNodes 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 TestNodes undergoes some post-processing, mainly to remove redundant Record wrappers.
+ * 3) The tree of TestNodes is converted into a TestSchema; it is walked recursively within this class.
+ */
+class TestWalker extends AbstractDSOMWalker {
+
+  // this is the critical data structure for managing the temporary TestNodes that are created
+  // when the Schema is initially walked.  This will then be converted to the actual TestSchema.
+  private var objectStack: List[TestNode] = List()
+
+  // this will contain the final TestSchema after step 3 as described above is complete.
+  private var result: Option[TestSchema] = None
+  def getResult: TestSchema = result.orNull
+  def stringRep: String = if (result.isDefined) result.get.toString else ""
+
+  /**
+   * Every event handler for onXBegin is routed to this helper method.  Individual properties of
+   * a given element are not actually handled here; these properties are extracted when a
+   * TestNode is instantiated
+   * @param dsomElement the particular element that was encountered.  Could be essentially any Term element.
+   */
+  private def onNodeBegin(dsomElement: TermView): Unit = {
+    val newNode: TestNode = new TestNode(dsomElement)
+    // we need add the new node as a new child of whatever is currently at the top of the stack
+    objectStack.head.addChild(newNode)
+    // however, we also add the node to the stack itself!  We need to be able to add children to it
+    // if it is, say, another record or array.
+    objectStack = newNode +: objectStack
+  }
+
+  /**
+   * Every event handler for onXEnd is routed to this helper method.  We don't need to do anything other
+   * than remove the top element of the stack; this should be the current element.
+   */
+  private def onNodeEnd(): Unit = objectStack = objectStack.tail
+
+  override def onTermBegin(termElement: TermView): Unit = termElement match {
+    case _: SequenceView | _: ChoiceView | _: ElementBaseView => onNodeBegin(termElement)
+    case _ =>
+  }
+
+  override def onTermEnd(termElement: TermView): Unit = termElement match {
+    case _: SequenceView | _: ChoiceView | _: ElementBaseView => onNodeEnd()
+    case _ =>
+  }
+
+  override def onTypeBegin(typeElement: TypeView): Unit = {}
+
+  override def onTypeEnd(typeElement: TypeView): Unit = {}
+
+  override def onWalkBegin(root: RootView): Unit = objectStack = List(new TestNode(root))
+
+  override def onWalkEnd(root: RootView): Unit = {
+    // After the walk is over, we perform postProcessing and then convert the TestNode tree
+    // into a TestSchema.  Also, if we are in dev. mode, we print out the TestNode tree
+    if (!RecordWalker.PRODUCTION_MODE) println(objectStack.head)
+    postProcessing()
+    if (!RecordWalker.PRODUCTION_MODE) println(objectStack.head)
+    result = Some(getRecordSchema(objectStack.head))
+  }
+
+  /**
+   * Determines if a Record is "extra"; that is, if it should be replaced with
+   * its list of children SchemaNodes within whatever parent TestNode it's a part of
+   * @param childNode the node to be considered
+   * @return true if this node is a Record and has no name, false otherwise
+   */
+  private def isExtraRecord(parentNode: TestNode, childNode: TestNode): Boolean = {
+    // any no-name nodes with no children are immediately removed
+    (childNode.name.isEmpty && childNode.children.isEmpty) || {
+      parentNode.fieldType match {
+        case RecordType | ArrayType =>
+          childNode.fieldType match {
+            // This removes extra wrapper records around children of records or arrays
+            // usually used to remove things like the DFDL complexType, simpleType elements
+            case RecordType => childNode.name.isEmpty
+            case _ => false
+          }
+        // Currently, all double choices are removed.  This was mainly done to make GroupRefs work
+        // for the JPEG Schema, but may not be the correct approach for all cases.
+        case ChoiceType =>
+          childNode.fieldType match {
+            case ChoiceType => true
+            case _ => false
+          }
+        case _ => false
+      }
+    }
+
+  }
+
+  /**
+   * Recursively replace any SchemaNodes that are of type record and do not have
+   * a name attribute with their children. These usually represent unnecessary wrapper nodes or empty
+   * records with no elements.
+   *
+   * Given a TestNode, we copy over all the children to a list, and initialize another empty list.
+   * Then, for every Node in the first list, if it is considered "extra",
+   * then its *children* are added to the second list.
+   * Otherwise, the Node itself is added.
+   *
+   * Then, we let the first list equal the second and we repeat this process
+   * until none of the children are "extra".
+   * This is necessary because sometimes we can have towers of extra Nodes that would
+   * never get resolved if we just took care of 1 or 2 layers; all must be dealt with at once.
+   *
+   * Then, the current TestNode's children list is replaced with this filtered one,
+   * and this method is called on all the items of that list.
+   * @param schemaNode the current Node undergoing the algorithm described above
+   */
+  private def removeExtraRecords(schemaNode: TestNode): Unit = {
+    schemaNode.fieldType match {
+      // Only SchemaNodes that are these 3 types should be able to have child nodes.
+      case RecordType | ChoiceType | ArrayType =>
+        if (schemaNode.children.exists(child => isExtraRecord(schemaNode, child))) {
+          var currChildren: ListBuffer[TestNode] = new ListBuffer[TestNode]()
+          for (child <- schemaNode.children) currChildren += child
+          while (currChildren.exists(child => isExtraRecord(schemaNode, child))) {
+            val toCopy: ListBuffer[TestNode] = new ListBuffer[TestNode]()
+            for (child <- currChildren) {
+              if (isExtraRecord(schemaNode, child)) {
+                for (subChild <- child.children) toCopy += subChild
+              } else toCopy += child
+            }
+            currChildren = toCopy
+          }
+          schemaNode.children = currChildren.toList
+        }
+      case _ =>
+    }
+    // call this helper method on each of this Nodes's children.  There is no infinite recursion because
+    // eventually a sub-child will not have any more children.  This of course assumes that we are dealing
+    // with a rooted tree (there shouldn't be any cycles), which should be the case.
+    for (child <- schemaNode.children) removeExtraRecords(child)
+  }
+
+  /**
+   * Perform postProcessing; this happens *after* the TestNode tree is created but *before* that tree
+   * gets converted to a TestSchema
+   */
+  private def postProcessing(): Unit = {
+    removeExtraRecords(objectStack.head)
+  }
+
+  private def getSimpleType(schemaNode: TestNode): FieldType = {
+    if (!schemaNode.isSimple) throw new IllegalArgumentException(s"Node $schemaNode is not a simple type!")
+    schemaNode.fieldType
+  }
+
+  private def nodeToField(schemaNode: TestNode): TestField = {
+    // by default, if this node doesn't have a name, its data type is used as the field name.
+    // This should only ever be the case for anonymous choices.
+    val recordField: TestField = new TestField(
+      schemaNode.name.getOrElse(schemaNode.fieldType.toString),
+      schemaNodeToDataType(schemaNode)
+    )
+    if (schemaNode.isOptional) new OptionalTestField(recordField) else recordField
+  }
+
+  private def newRecordSchema(schemaNode: TestNode, fieldList: List[TestField]): TestSchema = {
+    new TestSchema(schemaNode.name.getOrElse(""), schemaNode.namespace, fieldList)
+  }
+
+  /**
+   * Helper method to specifically convert a TestNode known to be a Record into a TestSchema.
+   * Each Child of the TestNode becomes a Field of the TestSchema
+   * @param schemaNode the node to convert to a TestSchema
+   * @return a tuple containing the finalized TestSchema and its corresponding Record Data Type
+   */
+  private def schemaNodeToRecordType(schemaNode: TestNode): RecordDataType = {
+    val fieldList: ListBuffer[TestField] = ListBuffer()
+    for (child <- schemaNode.children) fieldList += nodeToField(child)
+    val TestSchema: TestSchema = newRecordSchema(schemaNode, fieldList.toList)
+    RecordDataType(TestSchema)
+  }
+
+  /**
+   * Helper method to convert a TestNode known to be a choice into a Choice data type.
+   * This is able to handle a DFDL schema in which either named elements are directly sub-members of
+   * DFDL choices or if they are embedded in another element (which corresponds to being in a Record).
+   * In the end, if they are not already inside a Record, then they are put there. Choices cannot
+   * have fields, only possible sub-types, so anything that would be a "field" has a wrapper Record
+   * put around it.
+   * @param schemaNode the node to convert to a Choice data type
+   * @return a Choice data type as described above
+   */
+  private def choiceNodeToChoiceSchema(schemaNode: TestNode): DataType = {
+    val childList: ListBuffer[DataType] = new ListBuffer()
+    for (child <- schemaNode.children) {
+      if (child.name.isEmpty) {
+        childList += schemaNodeToRecordType(child)
+      } else {
+        // In the case where the child has a name, we embed it in an additional TestSchema,
+        // since we must add some DataType rather than a Field.
+        val fieldList: ListBuffer[TestField] = ListBuffer()
+        fieldList += nodeToField(child)
+        val choiceOption: TestSchema = newRecordSchema(child, fieldList.toList)
+        childList += RecordDataType(choiceOption)
+      }
+    }
+    ChoiceDataType(childList.toList)
+  }
+
+  /**
+   * Local helper method to appropriately convert a TestNode into an appropriate Record
+   * Data Type.  Records and Choices get routed to other helper methods, and Arrays are handled in the method.
+   * @param schemaNode the node to convert to a Record Data Type
+   * @return the finalized Record Data type
+   */
+  private def schemaNodeToDataType(schemaNode: TestNode): DataType = {
+    schemaNode.fieldType match {
+      case ArrayType =>
+        if (schemaNode.isSimple) {
+          ArrayDataType(new DataType(getSimpleType(schemaNode)))
+        } else if (schemaNode.children.nonEmpty) {
+          ArrayDataType(schemaNodeToRecordType(schemaNode))
+        } else {
+          throw new IllegalStateException(
+            s"Array Node $schemaNode either needs to be simple or have child nodes!"
+          )
+        }
+      case RecordType => schemaNodeToRecordType(schemaNode)
+      case ChoiceType => choiceNodeToChoiceSchema(schemaNode)
+      case _ => schemaNode.simpleType.get
+    }
+  }
+
+  /**
+   * In general, this method can be used to extract the TestSchema node from the tuple result
+   * of schemaNodeToRecordSchema.  However, it is really only used to kick off the conversion from
+   * the root TestNode.
+   * @param schemaNode the node from which a TestSchema will be extracted
+   * @return the processed TestSchema represented by the input
+   */
+  private def getRecordSchema(schemaNode: TestNode): TestSchema = {
+    schemaNodeToRecordType(schemaNode).childSchema
+  }
+
+}
+
+/**
+ * The sole purpose of this Companion Object is to contain this one static variable.
+ */
+object RecordWalker {
+  /**
+   * Used to suppress all of the extra printing from running test cases
+   */
+  final val PRODUCTION_MODE: Boolean = true

Review comment:
       I think I wound up with a solution that is similar to this idea; I had a testing file generate a Stack-like structure based on different View types it received, and the Tests were based around specific expected events in that List.  Since I was rewriting the Tests, I chose to automate as little as possible, I only automated the Simple Types since checking that they work was the same way for each one.  However, for each of the Complex types, I wanted to check different things like the name in some cases.  This is all in the latest revision.




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