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

[GitHub] [incubator-daffodil] mbeckerle commented on a change in pull request #422: WIP: Add runtime2 backend with C code generator

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



##########
File path: daffodil-cli/build.sbt
##########
@@ -42,6 +42,18 @@ mappings in Universal ++= Seq(
   baseDirectory.value / "README.md" -> "README.md",
 )
 
+mappings in Universal ++= Seq(
+  baseDirectory.value / ".." / "daffodil-runtime2" / "src" / "main" / "c" / "common_runtime.h" -> "include/common_runtime.h",
+  baseDirectory.value / ".." / "daffodil-runtime2" / "src" / "main" / "c" / "daffodil_argp.h" -> "include/daffodil_argp.h",
+  baseDirectory.value / ".." / "daffodil-runtime2" / "src" / "main" / "c" / "stack.h" -> "include/stack.h",
+  baseDirectory.value / ".." / "daffodil-runtime2" / "src" / "main" / "c" / "xml_reader.h" -> "include/xml_reader.h",
+  baseDirectory.value / ".." / "daffodil-runtime2" / "src" / "main" / "c" / "xml_writer.h" -> "include/xml_writer.h",
+)
+
+mappings in Universal ++= Seq(
+  baseDirectory.value / ".." / "daffodil-runtime2" / "target" / "streams" / "compile" / "ccTargetMap" / "_global" / "streams" / "compile" / "sbtcc.Library" / "libruntime2.a" -> "lib/libruntime2.a",

Review comment:
       Hmmm. What does this line do? It mentions "target" explicitly, which is very odd to me. 

##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/runtime2/generators/ParserGenerator.scala
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.runtime2.generators
+
+import org.apache.daffodil.api.DFDL
+import org.apache.daffodil.dpath.NodeInfo
+import org.apache.daffodil.dpath.NodeInfo.PrimType
+import org.apache.daffodil.dsom.ElementBase
+import org.apache.daffodil.exceptions.ThrowsSDE
+
+import scala.collection.mutable
+
+/**
+ * Gives an object the ability to generate code.
+ */
+trait ParserGenerator {
+  def generateCode(state: CodeGeneratorState): Unit
+}
+
+/**
+ * Builds up the state of generated code.
+ */
+class CodeGeneratorState extends DFDL.CodeGeneratorState {
+  private val structs = mutable.Stack[ComplexCGState]()
+  private val prototypes = mutable.ArrayBuffer[String]()
+  private val erds = mutable.ArrayBuffer[String]()
+  private val finalStructs = mutable.ArrayBuffer[String]()
+  private val finalImplementation = mutable.ArrayBuffer[String]()
+
+  def addImplementation(context: ElementBase): Unit = {
+    val C = context.namedQName.local
+    val initStatements = structs.top.initStatements.mkString("\n")
+    val parserStatements = structs.top.parserStatements.mkString("\n")
+    val unparserStatements = structs.top.unparserStatements.mkString("\n")
+    val prototypeFunctions =
+      s"""static void        ${C}_initSelf($C *instance);
+         |static const char *${C}_parseSelf($C *instance, const PState *pstate);
+         |static const char *${C}_unparseSelf(const $C *instance, const UState *ustate);""".stripMargin
+    prototypes += prototypeFunctions
+    val functions =
+      s"""static void
+         |${C}_initSelf($C *instance)
+         |{
+         |$initStatements
+         |}
+         |
+         |static const char *
+         |${C}_parseSelf($C *instance, const PState *pstate)
+         |{
+         |    const char *error_msg = NULL;
+         |$parserStatements
+         |    return error_msg;
+         |}
+         |
+         |static const char *
+         |${C}_unparseSelf(const $C *instance, const UState *ustate)
+         |{
+         |    const char *error_msg = NULL;
+         |$unparserStatements
+         |    return error_msg;
+         |}
+         |""".stripMargin
+    finalImplementation += functions
+  }
+
+  private def defineQNameInit(context: ElementBase): String = {
+    val qname = context.namedQName.toQNameString
+    val xmlns = if (context.namedQName.prefix.isDefined) s"xmlns:${context.namedQName.prefix.get}" else "xmlns"
+    val ns = context.namedQName.namespace.toStringOrNullIfNoNS
+    // Optimize away xmlns=ns declaration if possible, although this approach may not be entirely correct
+    val parentOpt = context.enclosingElements.headOption
+    val parentNs = if (parentOpt.isDefined) parentOpt.get.namedQName.namespace.toStringOrNullIfNoNS
+    val qnameInit = if (ns == null || ns == parentNs)
+      s"""    {"$qname"},       // namedQName.name"""
+    else
+      s"""    {
+         |        "$qname",              // namedQName.name
+         |        "$xmlns",           // namedQName.xmlns
+         |        "$ns", // namedQName.ns
+         |    },""".stripMargin
+    qnameInit
+  }
+
+  def addComplexTypeERD(context: ElementBase): Unit = {
+    val C = context.namedQName.local
+    val count = structs.top.declarations.length
+    val offsetComputations = structs.top.offsetComputations.mkString(",\n")
+    val erdComputations = structs.top.erdComputations.mkString(",\n")
+    val qnameInit = defineQNameInit(context)
+    val complexERD =
+      s"""static const $C ${C}_compute_ERD_offsets;
+         |
+         |static const ptrdiff_t ${C}_offsets[$count] = {
+         |$offsetComputations
+         |};
+         |
+         |static const ERD *${C}_childrenERDs[$count] = {
+         |$erdComputations
+         |};
+         |
+         |static const ERD ${C}_ERD = {
+         |$qnameInit
+         |    COMPLEX,                         // typeCode
+         |    $count,                               // numChildren
+         |    ${C}_offsets,                      // offsets
+         |    ${C}_childrenERDs,                 // childrenERDs
+         |    (ERDInitSelf)&${C}_initSelf,       // initSelf
+         |    (ERDParseSelf)&${C}_parseSelf,     // parseSelf
+         |    (ERDUnparseSelf)&${C}_unparseSelf, // unparseSelf
+         |};
+         |""".stripMargin
+    erds += complexERD
+  }
+
+  def addStruct(context: ElementBase): Unit = {
+    val C = context.namedQName.local
+    val declarations = structs.top.declarations.mkString("\n")
+    val struct =
+      s"""typedef struct $C
+         |{
+         |    InfosetBase _base;
+         |$declarations
+         |} $C;
+         |""".stripMargin
+    finalStructs += struct
+    val initStatement = s"    instance->_base.erd = &${C}_ERD;"
+    structs.top.initStatements += initStatement
+  }
+
+  def addSimpleTypeStatements(initStatement: String, parseStatement: String, unparseStatement: String): Unit = {
+    structs.top.initStatements += initStatement
+    structs.top.parserStatements += parseStatement
+    structs.top.unparserStatements += unparseStatement
+  }
+
+  def addComplexTypeStatements(child: ElementBase): Unit = {
+    val C = child.namedQName.local
+    val e = child.name
+    val initStatement = s"    ${C}_initSelf(&instance->$e);"
+    val parseStatement =
+      s"""    if (error_msg == NULL)
+         |    {
+         |        error_msg = ${C}_parseSelf(&instance->$e, pstate);
+         |    }""".stripMargin
+    val unparseStatement =
+      s"""    if (error_msg == NULL)
+         |    {
+         |        error_msg = ${C}_unparseSelf(&instance->$e, ustate);
+         |    }""".stripMargin
+    structs.top.initStatements += initStatement
+    structs.top.parserStatements += parseStatement
+    structs.top.unparserStatements += unparseStatement
+  }
+
+  def pushComplexElement(context: ElementBase): Unit = {
+    val C = context.namedQName.local
+    structs.push(new ComplexCGState(C))
+  }
+
+  def popComplexElement(context: ElementBase): Unit = {
+    structs.pop()
+  }
+
+  def addSimpleTypeERD(context: ElementBase): Unit = {
+    val e = context.namedQName.local
+    val qnameInit = defineQNameInit(context)
+    val typeCode = context.optPrimType.get match {
+      case PrimType.Int => "PRIMITIVE_INT32"
+      case PrimType.String => "PRIMITIVE_STRING"
+      case p: PrimType => context.SDE("PrimType %s not supported yet.", p.toString)
+    }
+    val erd =
+      s"""static const ERD ${e}_ERD = {
+         |$qnameInit
+         |    $typeCode, // typeCode
+         |    0,               // numChildren
+         |    NULL,            // offsets
+         |    NULL,            // childrenERDs
+         |    NULL,            // initSelf
+         |    NULL,            // parseSelf
+         |    NULL,            // unparseSelf
+         |};
+         |""".stripMargin
+    erds += erd
+    addComputations(context)
+  }
+
+  def addComputations(child: ElementBase): Unit = {
+    val C = structs.top.C
+    val e = child.namedQName.local
+    val offsetComputation = s"    (char *)&${C}_compute_ERD_offsets.$e - (char *)&${C}_compute_ERD_offsets"
+    val erdComputation = s"    &${e}_ERD"
+    structs.top.offsetComputations += offsetComputation
+    structs.top.erdComputations += erdComputation
+  }
+
+  def addFieldDeclaration(context: ThrowsSDE, child: ElementBase): Unit = {
+    val definition = if (child.isSimpleType) {
+      import NodeInfo.PrimType
+      child.optPrimType.get match {
+        case PrimType.Long => "int64_t    "
+        case PrimType.Int => "int32_t    "
+        case x => context.SDE("Unsupported primitive type: " + x)
+      }
+    } else {
+      child.namedQName.local + "         "
+    }
+    structs.top.declarations += s"    $definition ${child.name};"
+  }
+
+  def viewCodeHeader: String = {

Review comment:
       Why are these named with "view" prefix? Seems like "gen" or "generate" would be better. 

##########
File path: daffodil-runtime2/src/main/c/xml_reader.c
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.
+ */
+
+#include "xml_reader.h"
+#include <errno.h>  // for errno, ERANGE
+#include <limits.h> // for LONG_MAX, LONG_MIN
+#include <mxml.h>   // for mxmlWalkNext, mxmlGetElement, mxmlGetType, ...
+#include <stdint.h> // for int32_t, INT32_MAX, INT32_MIN
+#include <stdlib.h> // for NULL, strtol
+#include <string.h> // for strcmp, strerror
+
+// Read XML data from file before walking infoset
+
+static const char *
+xmlStartDocument(XMLReader *reader)
+{
+    // Load the XML data into memory
+    reader->xml = mxmlLoadFile(NULL, reader->stream, MXML_OPAQUE_CALLBACK);
+    reader->node = reader->xml;
+    if (reader->node == NULL) {
+        return "Unable to read XML data from input file";
+    }
+
+    // Consume the <?xml line if there is one
+    const char *name = mxmlGetElement(reader->node);
+    if (name && strncmp(name, "?xml", strlen("?xml")) == 0) {
+        do
+        {
+            reader->node = mxmlWalkNext(reader->node, reader->xml, MXML_DESCEND);
+        } while (mxmlGetType(reader->node) == MXML_OPAQUE);
+        name = mxmlGetElement(reader->node);
+    }
+
+    // Consume a comment if there is one
+    if (name && strncmp(name, "!--", strlen("!--")) == 0) {
+        do
+        {
+            reader->node = mxmlWalkNext(reader->node, reader->xml, MXML_DESCEND);
+        } while (mxmlGetType(reader->node) == MXML_OPAQUE);
+    }
+
+    return reader->node ? NULL : "Ran out of XML data";
+}
+
+// Delete XML data after walking infoset
+
+static const char *
+xmlEndDocument(XMLReader *reader)
+{
+    // Consume any remaining newlines or whitespace
+    while (mxmlGetType(reader->node) == MXML_OPAQUE) {
+        reader->node = mxmlWalkNext(reader->node, reader->xml, MXML_DESCEND);
+    }
+
+    // Check whether we have consumed all of the XML data
+    if (reader->node != NULL) {
+        // This code path exits the program - no need to call mxmlDelete
+        return "Did not consume all of the XML data";
+    }
+
+    // Free the storage allocated to hold the XML data
+    mxmlDelete(reader->xml);
+    reader->xml = NULL;
+    reader->node = NULL;
+    return NULL;
+}
+
+// Continue walking both XML data and infoset in lockstep
+
+static const char *
+xmlStartComplex(XMLReader *reader, const InfosetBase *base)
+{
+    // Consume any newlines or whitespace before the element
+    while (mxmlGetType(reader->node) == MXML_OPAQUE) {
+        reader->node = mxmlWalkNext(reader->node, reader->xml, MXML_DESCEND);
+    }
+
+    // Get the element and consume it
+    const char *name_from_xml = mxmlGetElement(reader->node);
+    const char *name_from_infoset = base->erd->namedQName.name;
+    reader->node = mxmlWalkNext(reader->node, reader->xml, MXML_DESCEND);
+
+    // Check whether we are walking both XML data and infoset in lockstep
+    if (name_from_xml && name_from_infoset)
+    {
+        return strcmp(name_from_xml, name_from_infoset) == 0
+                   ? NULL
+                   : "Found mismatch between XML data and infoset";
+    }
+    else
+    {
+        return "Ran out of XML data";
+    }
+}
+
+// Consume XML data only on start events, not end events
+
+static const char *
+xmlEndComplex(XMLReader *reader, const InfosetBase *base)
+{
+    (void)reader;
+    (void)base;
+    return NULL;
+}
+
+// Read 32-bit integer value from XML data
+
+static const char *
+xmlInt32Elem(XMLReader *reader, const ERD *erd, int32_t *location)
+{
+    // Consume any newlines or whitespace before the element
+    while (mxmlGetType(reader->node) == MXML_OPAQUE) {
+        reader->node = mxmlWalkNext(reader->node, reader->xml, MXML_DESCEND);
+    }
+
+    // Get the element and consume it
+    const char *name_from_xml = mxmlGetElement(reader->node);
+    const char *number_from_xml = mxmlGetOpaque(reader->node);
+    const char *name_from_infoset = erd->namedQName.name;
+    reader->node = mxmlWalkNext(reader->node, reader->xml, MXML_DESCEND);

Review comment:
       here too, shouldn't this check be before the walk call?

##########
File path: build.sbt
##########
@@ -43,6 +46,32 @@ lazy val runtime1         = Project("daffodil-runtime1", file("daffodil-runtime1
                               .dependsOn(io, lib % "test->test", udf, macroLib % "compile-internal, test-internal")
                               .settings(commonSettings, usesMacros)
 
+val runtime2StaticLib     = Library("libruntime2.a")
+lazy val runtime2         = Project("daffodil-runtime2", file("daffodil-runtime2")).configs(IntegrationTest)
+                              .enablePlugins(CcPlugin)
+                              .dependsOn(tdmlProc)
+                              .settings(commonSettings)
+                              .settings(publishArtifact in (Compile, packageDoc) := false)
+                              .settings(
+                                Compile / ccTargets := ListSet(runtime2StaticLib),
+                                Compile / cSources  := Map(
+                                  runtime2StaticLib -> Seq(
+                                    baseDirectory.value / "src" / "main" / "c" / "common_runtime.c",

Review comment:
       It seems surprising to me that we have to list every file here that goes into the static lib. We have to maintain this, or perhaps we create a subdir and rules about packaging the C code found in that subdir? Or maybe runtime2StaticSlib is a separate subproject from the rest of runtime2 so that all its files can go in the static library? 
   
   I'm not suggesting we fix this right now. It's just something to think about because maintaining this will be a pain. 

##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/grammar/primitives/SequenceCombinator.scala
##########
@@ -91,6 +92,16 @@ class OrderedSequence(sq: SequenceTermBase, sequenceChildrenArg: Seq[SequenceChi
       }
     }
   }
+
+  override def generateCode(cgState: CodeGeneratorState): Unit = {
+    //
+    // To lift this draconian restriction, we have to
+    // generate code for each of the children, and combine them into a block
+    //
+    sq.schemaDefinitionUnless(sequenceChildren.length == 1, "Only a single child of a sequence is supported.")

Review comment:
       Oh, I thought this was taken out and we had tests that have nested complex types, the leaf simple types of which are the 32-bit ints which are the only thing we support. 

##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/grammar/primitives/ElementCombinator.scala
##########
@@ -280,8 +300,11 @@ case class CaptureValueLengthEnd(ctxt: ElementBase)
       new CaptureEndOfValueLengthUnparser(ctxt.erd)
     else
       new NadaUnparser(ctxt.erd)
-}
 
+  override def generateCode(state: CodeGeneratorState): Unit = {
+    // Not generating code here

Review comment:
       These are not yet supported. But rather than issuing an SDE here, since they get put into the grammar tree (probably by mistake - missed optimization) we need for now for these to not fail/SDE. 
   
   I think these should SDW saying not supported. When we fix the bug that has these things in the grammar at all, then the warnings would go away. 
   
   I realize the base method being overridden does a generic "not supported" SDE. 

##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/runtime2/GeneratedCodeCompiler.scala
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.runtime2
+
+import org.apache.commons.io.FileUtils
+import org.apache.daffodil.compiler.ProcessorFactory
+import org.apache.daffodil.dsom.SchemaDefinitionError
+import org.apache.daffodil.runtime2.generators.CodeGeneratorState
+import org.apache.daffodil.util.Misc
+import os.Pipe
+
+class GeneratedCodeCompiler(pf: ProcessorFactory) {
+  private var executableFile: os.Path = _
+  private lazy val isWindows = System.getProperty("os.name").toLowerCase().startsWith("windows")
+
+  /**
+   * Compiles and links generated C code with runtime2 library to
+   * build an executable file.
+   */
+  def compile(rootElementName: String, codeGeneratorState: CodeGeneratorState): Unit = {
+    val compiler = "cc"
+    val location = Option(this.getClass.getProtectionDomain.getCodeSource) flatMap (x => Option(x.getLocation))
+    val wd = if (os.exists(os.pwd/"daffodil-runtime2"))
+      os.pwd
+    else if (os.exists(os.pwd/os.up/"daffodil-runtime2"))
+      os.pwd/os.up
+    else if (location.isDefined)
+      os.Path(FileUtils.toFile(location.get))/os.up/os.up
+    else
+      os.pwd
+    val includeDir = if (os.exists(wd/"include"))
+      wd/"include"
+    else
+      wd/"daffodil-runtime2"/"src"/"main"/"c"
+    val libDir = if (os.exists(wd/"lib"))
+      wd/"lib"
+    else
+      wd/"daffodil-runtime2"/"target"/"streams"/"compile"/"ccTargetMap"/"_global"/"streams"/"compile"/"sbtcc.Library"

Review comment:
       Shouldn't the sbt-cc plugin take care of this sort of thing? Do we need this so we can TDML test? 

##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/runtime2/generators/ParserGenerator.scala
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.runtime2.generators
+
+import org.apache.daffodil.api.DFDL
+import org.apache.daffodil.dpath.NodeInfo
+import org.apache.daffodil.dpath.NodeInfo.PrimType
+import org.apache.daffodil.dsom.ElementBase
+import org.apache.daffodil.exceptions.ThrowsSDE
+
+import scala.collection.mutable
+
+/**
+ * Gives an object the ability to generate code.
+ */
+trait ParserGenerator {
+  def generateCode(state: CodeGeneratorState): Unit
+}
+
+/**
+ * Builds up the state of generated code.
+ */
+class CodeGeneratorState extends DFDL.CodeGeneratorState {
+  private val structs = mutable.Stack[ComplexCGState]()
+  private val prototypes = mutable.ArrayBuffer[String]()
+  private val erds = mutable.ArrayBuffer[String]()
+  private val finalStructs = mutable.ArrayBuffer[String]()
+  private val finalImplementation = mutable.ArrayBuffer[String]()
+
+  def addImplementation(context: ElementBase): Unit = {
+    val C = context.namedQName.local
+    val initStatements = structs.top.initStatements.mkString("\n")
+    val parserStatements = structs.top.parserStatements.mkString("\n")
+    val unparserStatements = structs.top.unparserStatements.mkString("\n")
+    val prototypeFunctions =
+      s"""static void        ${C}_initSelf($C *instance);
+         |static const char *${C}_parseSelf($C *instance, const PState *pstate);
+         |static const char *${C}_unparseSelf(const $C *instance, const UState *ustate);""".stripMargin
+    prototypes += prototypeFunctions
+    val functions =
+      s"""static void
+         |${C}_initSelf($C *instance)
+         |{
+         |$initStatements
+         |}
+         |
+         |static const char *
+         |${C}_parseSelf($C *instance, const PState *pstate)
+         |{
+         |    const char *error_msg = NULL;
+         |$parserStatements
+         |    return error_msg;
+         |}
+         |
+         |static const char *
+         |${C}_unparseSelf(const $C *instance, const UState *ustate)
+         |{
+         |    const char *error_msg = NULL;
+         |$unparserStatements
+         |    return error_msg;
+         |}
+         |""".stripMargin
+    finalImplementation += functions
+  }
+
+  private def defineQNameInit(context: ElementBase): String = {
+    val qname = context.namedQName.toQNameString
+    val xmlns = if (context.namedQName.prefix.isDefined) s"xmlns:${context.namedQName.prefix.get}" else "xmlns"
+    val ns = context.namedQName.namespace.toStringOrNullIfNoNS
+    // Optimize away xmlns=ns declaration if possible, although this approach may not be entirely correct
+    val parentOpt = context.enclosingElements.headOption
+    val parentNs = if (parentOpt.isDefined) parentOpt.get.namedQName.namespace.toStringOrNullIfNoNS
+    val qnameInit = if (ns == null || ns == parentNs)

Review comment:
       Best not to convert things into null pointers or not. 
   You have Option types here. Should use those. 
   E.g.,
   ```
   val optXMLPrefixDefinition = context.namedQName.prefix.map{ pre => s"xmlns:${pre}"}
   ```
   I think there's 2 possibilities here: no namespace, namespace 
   Then within namespace there's 2 possibilities: with prefix or default namespace. 
   So I think this can be simplified, and have no possible null pointer exceptions.
   

##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/runtime2/generators/ParserGenerator.scala
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.runtime2.generators
+
+import org.apache.daffodil.api.DFDL
+import org.apache.daffodil.dpath.NodeInfo
+import org.apache.daffodil.dpath.NodeInfo.PrimType
+import org.apache.daffodil.dsom.ElementBase
+import org.apache.daffodil.exceptions.ThrowsSDE
+
+import scala.collection.mutable
+
+/**
+ * Gives an object the ability to generate code.
+ */
+trait ParserGenerator {
+  def generateCode(state: CodeGeneratorState): Unit
+}
+
+/**
+ * Builds up the state of generated code.
+ */
+class CodeGeneratorState extends DFDL.CodeGeneratorState {
+  private val structs = mutable.Stack[ComplexCGState]()
+  private val prototypes = mutable.ArrayBuffer[String]()
+  private val erds = mutable.ArrayBuffer[String]()
+  private val finalStructs = mutable.ArrayBuffer[String]()
+  private val finalImplementation = mutable.ArrayBuffer[String]()
+
+  def addImplementation(context: ElementBase): Unit = {
+    val C = context.namedQName.local
+    val initStatements = structs.top.initStatements.mkString("\n")
+    val parserStatements = structs.top.parserStatements.mkString("\n")
+    val unparserStatements = structs.top.unparserStatements.mkString("\n")
+    val prototypeFunctions =
+      s"""static void        ${C}_initSelf($C *instance);
+         |static const char *${C}_parseSelf($C *instance, const PState *pstate);
+         |static const char *${C}_unparseSelf(const $C *instance, const UState *ustate);""".stripMargin
+    prototypes += prototypeFunctions
+    val functions =
+      s"""static void
+         |${C}_initSelf($C *instance)
+         |{
+         |$initStatements
+         |}
+         |
+         |static const char *
+         |${C}_parseSelf($C *instance, const PState *pstate)
+         |{
+         |    const char *error_msg = NULL;
+         |$parserStatements
+         |    return error_msg;
+         |}
+         |
+         |static const char *
+         |${C}_unparseSelf(const $C *instance, const UState *ustate)
+         |{
+         |    const char *error_msg = NULL;
+         |$unparserStatements
+         |    return error_msg;
+         |}
+         |""".stripMargin
+    finalImplementation += functions
+  }
+
+  private def defineQNameInit(context: ElementBase): String = {
+    val qname = context.namedQName.toQNameString
+    val xmlns = if (context.namedQName.prefix.isDefined) s"xmlns:${context.namedQName.prefix.get}" else "xmlns"
+    val ns = context.namedQName.namespace.toStringOrNullIfNoNS
+    // Optimize away xmlns=ns declaration if possible, although this approach may not be entirely correct
+    val parentOpt = context.enclosingElements.headOption
+    val parentNs = if (parentOpt.isDefined) parentOpt.get.namedQName.namespace.toStringOrNullIfNoNS
+    val qnameInit = if (ns == null || ns == parentNs)
+      s"""    {"$qname"},       // namedQName.name"""
+    else
+      s"""    {
+         |        "$qname",              // namedQName.name
+         |        "$xmlns",           // namedQName.xmlns
+         |        "$ns", // namedQName.ns
+         |    },""".stripMargin
+    qnameInit
+  }
+
+  def addComplexTypeERD(context: ElementBase): Unit = {
+    val C = context.namedQName.local
+    val count = structs.top.declarations.length
+    val offsetComputations = structs.top.offsetComputations.mkString(",\n")
+    val erdComputations = structs.top.erdComputations.mkString(",\n")
+    val qnameInit = defineQNameInit(context)
+    val complexERD =
+      s"""static const $C ${C}_compute_ERD_offsets;
+         |
+         |static const ptrdiff_t ${C}_offsets[$count] = {
+         |$offsetComputations
+         |};
+         |
+         |static const ERD *${C}_childrenERDs[$count] = {
+         |$erdComputations
+         |};
+         |
+         |static const ERD ${C}_ERD = {
+         |$qnameInit
+         |    COMPLEX,                         // typeCode
+         |    $count,                               // numChildren
+         |    ${C}_offsets,                      // offsets
+         |    ${C}_childrenERDs,                 // childrenERDs
+         |    (ERDInitSelf)&${C}_initSelf,       // initSelf
+         |    (ERDParseSelf)&${C}_parseSelf,     // parseSelf
+         |    (ERDUnparseSelf)&${C}_unparseSelf, // unparseSelf
+         |};
+         |""".stripMargin
+    erds += complexERD
+  }
+
+  def addStruct(context: ElementBase): Unit = {
+    val C = context.namedQName.local
+    val declarations = structs.top.declarations.mkString("\n")
+    val struct =
+      s"""typedef struct $C
+         |{
+         |    InfosetBase _base;
+         |$declarations
+         |} $C;
+         |""".stripMargin
+    finalStructs += struct
+    val initStatement = s"    instance->_base.erd = &${C}_ERD;"
+    structs.top.initStatements += initStatement
+  }
+
+  def addSimpleTypeStatements(initStatement: String, parseStatement: String, unparseStatement: String): Unit = {
+    structs.top.initStatements += initStatement
+    structs.top.parserStatements += parseStatement
+    structs.top.unparserStatements += unparseStatement
+  }
+
+  def addComplexTypeStatements(child: ElementBase): Unit = {
+    val C = child.namedQName.local
+    val e = child.name
+    val initStatement = s"    ${C}_initSelf(&instance->$e);"
+    val parseStatement =
+      s"""    if (error_msg == NULL)
+         |    {
+         |        error_msg = ${C}_parseSelf(&instance->$e, pstate);
+         |    }""".stripMargin
+    val unparseStatement =
+      s"""    if (error_msg == NULL)
+         |    {
+         |        error_msg = ${C}_unparseSelf(&instance->$e, ustate);
+         |    }""".stripMargin
+    structs.top.initStatements += initStatement
+    structs.top.parserStatements += parseStatement
+    structs.top.unparserStatements += unparseStatement
+  }
+
+  def pushComplexElement(context: ElementBase): Unit = {
+    val C = context.namedQName.local
+    structs.push(new ComplexCGState(C))
+  }
+
+  def popComplexElement(context: ElementBase): Unit = {
+    structs.pop()
+  }
+
+  def addSimpleTypeERD(context: ElementBase): Unit = {
+    val e = context.namedQName.local
+    val qnameInit = defineQNameInit(context)
+    val typeCode = context.optPrimType.get match {
+      case PrimType.Int => "PRIMITIVE_INT32"
+      case PrimType.String => "PRIMITIVE_STRING"
+      case p: PrimType => context.SDE("PrimType %s not supported yet.", p.toString)
+    }
+    val erd =
+      s"""static const ERD ${e}_ERD = {
+         |$qnameInit
+         |    $typeCode, // typeCode
+         |    0,               // numChildren
+         |    NULL,            // offsets
+         |    NULL,            // childrenERDs
+         |    NULL,            // initSelf
+         |    NULL,            // parseSelf
+         |    NULL,            // unparseSelf
+         |};
+         |""".stripMargin
+    erds += erd
+    addComputations(context)
+  }
+
+  def addComputations(child: ElementBase): Unit = {
+    val C = structs.top.C
+    val e = child.namedQName.local
+    val offsetComputation = s"    (char *)&${C}_compute_ERD_offsets.$e - (char *)&${C}_compute_ERD_offsets"
+    val erdComputation = s"    &${e}_ERD"
+    structs.top.offsetComputations += offsetComputation
+    structs.top.erdComputations += erdComputation
+  }
+
+  def addFieldDeclaration(context: ThrowsSDE, child: ElementBase): Unit = {
+    val definition = if (child.isSimpleType) {
+      import NodeInfo.PrimType
+      child.optPrimType.get match {
+        case PrimType.Long => "int64_t    "
+        case PrimType.Int => "int32_t    "
+        case x => context.SDE("Unsupported primitive type: " + x)
+      }
+    } else {
+      child.namedQName.local + "         "
+    }
+    structs.top.declarations += s"    $definition ${child.name};"
+  }
+
+  def viewCodeHeader: String = {
+    val structs = finalStructs.mkString("\n")
+    val header =
+      s"""#ifndef GENERATED_CODE_H
+         |#define GENERATED_CODE_H
+         |
+         |#include "common_runtime.h" // for InfosetBase
+         |#include <stdint.h>         // for int32_t
+         |
+         |// Return the root of an infoset to be used for parsing or unparsing
+         |
+         |extern InfosetBase *rootInfoset();
+         |
+         |// Define some infoset structures
+         |
+         |$structs
+         |#endif // GENERATED_CODE_H
+         |""".stripMargin
+    header
+  }
+
+  def viewCodeFile(rootElementName: String): String = {
+    val prototypes = this.prototypes.mkString("\n")
+    val erds = this.erds.mkString("\n")
+    val finalImplementation = this.finalImplementation.mkString("\n")
+    val code =
+      s"""#include "generated_code.h"
+         |#include <endian.h> // for be32toh, htobe32
+         |#include <errno.h>  // for errno
+         |#include <stddef.h> // for ptrdiff_t
+         |#include <stdio.h>  // for NULL, fread, fwrite, size_t, feof, ferror, FILE
+         |#include <string.h> // for strerror
+         |
+         |// Prototypes needed for compilation
+         |
+         |$prototypes
+         |
+         |// Metadata singletons
+         |
+         |$erds
+         |// Return the root of an infoset to be used for parsing or unparsing
+         |
+         |InfosetBase *
+         |rootInfoset()
+         |{
+         |    static $rootElementName    instance;
+         |    InfosetBase *root = &instance._base;
+         |    ${rootElementName}_ERD.initSelf(root);
+         |    return root;
+         |}
+         |
+         |// Methods to initialize, parse, and unparse infoset nodes
+         |
+         |static const char *

Review comment:
       This eof_or_error_msg can be static code in libruntime2.a. It has no substitutions going into it. 
   However, it's possible the string should get the current element name appended into it, but if that's the case it would also need the function name to be element specific, and the call to it from the parseSelf and unparseSelf would have to take that into consideration. 
   
   Maybe leave it generic static code (and put in library)  but pass both the FILE* and the ERD to it, and it can get the element name/context info from the ERD?

##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/runtime2/generators/ParserGenerator.scala
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.runtime2.generators
+
+import org.apache.daffodil.api.DFDL
+import org.apache.daffodil.dpath.NodeInfo
+import org.apache.daffodil.dpath.NodeInfo.PrimType
+import org.apache.daffodil.dsom.ElementBase
+import org.apache.daffodil.exceptions.ThrowsSDE
+
+import scala.collection.mutable
+
+/**
+ * Gives an object the ability to generate code.
+ */
+trait ParserGenerator {
+  def generateCode(state: CodeGeneratorState): Unit
+}
+
+/**
+ * Builds up the state of generated code.
+ */
+class CodeGeneratorState extends DFDL.CodeGeneratorState {
+  private val structs = mutable.Stack[ComplexCGState]()
+  private val prototypes = mutable.ArrayBuffer[String]()
+  private val erds = mutable.ArrayBuffer[String]()
+  private val finalStructs = mutable.ArrayBuffer[String]()
+  private val finalImplementation = mutable.ArrayBuffer[String]()
+
+  def addImplementation(context: ElementBase): Unit = {
+    val C = context.namedQName.local
+    val initStatements = structs.top.initStatements.mkString("\n")
+    val parserStatements = structs.top.parserStatements.mkString("\n")
+    val unparserStatements = structs.top.unparserStatements.mkString("\n")
+    val prototypeFunctions =
+      s"""static void        ${C}_initSelf($C *instance);
+         |static const char *${C}_parseSelf($C *instance, const PState *pstate);
+         |static const char *${C}_unparseSelf(const $C *instance, const UState *ustate);""".stripMargin
+    prototypes += prototypeFunctions
+    val functions =
+      s"""static void
+         |${C}_initSelf($C *instance)
+         |{
+         |$initStatements
+         |}
+         |
+         |static const char *
+         |${C}_parseSelf($C *instance, const PState *pstate)
+         |{
+         |    const char *error_msg = NULL;
+         |$parserStatements
+         |    return error_msg;
+         |}
+         |
+         |static const char *
+         |${C}_unparseSelf(const $C *instance, const UState *ustate)
+         |{
+         |    const char *error_msg = NULL;
+         |$unparserStatements
+         |    return error_msg;
+         |}
+         |""".stripMargin
+    finalImplementation += functions
+  }
+
+  private def defineQNameInit(context: ElementBase): String = {
+    val qname = context.namedQName.toQNameString
+    val xmlns = if (context.namedQName.prefix.isDefined) s"xmlns:${context.namedQName.prefix.get}" else "xmlns"
+    val ns = context.namedQName.namespace.toStringOrNullIfNoNS
+    // Optimize away xmlns=ns declaration if possible, although this approach may not be entirely correct
+    val parentOpt = context.enclosingElements.headOption
+    val parentNs = if (parentOpt.isDefined) parentOpt.get.namedQName.namespace.toStringOrNullIfNoNS
+    val qnameInit = if (ns == null || ns == parentNs)
+      s"""    {"$qname"},       // namedQName.name"""
+    else
+      s"""    {
+         |        "$qname",              // namedQName.name
+         |        "$xmlns",           // namedQName.xmlns

Review comment:
       So the qname initializer either is just one string, or it is 3 strings. The qname structure always has 3 fields. If the initializer is short, are the other members initialized to null, or just left random? We need them to be null. 
   
   Also, it appears the string "xmlns" in the 2nd member of this struct means "there is a namespace, but no prefix for it". I would use string "default namespace" for that to be more explicit, or just null. 

##########
File path: daffodil-core/src/test/scala/org/apache/daffodil/runtime2/TestGeneratedCodeCompiler.scala
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.runtime2
+
+import java.io.ByteArrayInputStream
+
+import org.apache.daffodil.compiler.Compiler
+import org.apache.daffodil.infoset.DIComplex
+import org.apache.daffodil.infoset.TestInfoset
+import org.apache.daffodil.util.Misc
+import org.apache.daffodil.util.SchemaUtils
+import org.junit.Test
+
+// Now that we can run TDML tests with runtime2, this test's remaining
+// value is for debugging of runtime2 components.
+class TestGeneratedCodeCompiler {
+
+  @Test
+  def compileRunParseInt32(): Unit = {
+    // Compile a DFDL schema to parse int32 numbers
+    val testSchema = SchemaUtils.dfdlTestSchema(
+        <xs:include schemaLocation="org/apache/daffodil/xsd/DFDLGeneralFormat.dfdl.xsd"/>,
+        <dfdl:format representation="binary" ref="GeneralFormat"/>,
+      <xs:element name="c1">
+        <xs:complexType>
+          <xs:sequence>
+            <xs:element name="e1" type="xs:int"/>
+            <xs:element name="c2">
+              <xs:complexType>
+                <xs:sequence>
+                  <xs:element name="e2" type="xs:int"/>

Review comment:
       So here we have a complex type sequence of two int. But above there's a comment where there is code that insists on only one child for a sequence. 
   
   Those are not consistent. If this test works, that code above must be dead code. 
   
   ???

##########
File path: daffodil-runtime1/src/main/scala/org/apache/daffodil/api/DFDLParserUnparser.scala
##########
@@ -202,6 +204,14 @@ object DFDL {
     def parse(ab: Array[Byte]): Unit
   }
 
+  trait CodeGeneratorState {

Review comment:
       I don't think we need this. This state object is state of the compiler/generator that emits C code. That's part of daffodil core, not daffodil-runtime2. 

##########
File path: daffodil-core/src/test/scala/org/apache/daffodil/runtime2/TestGeneratedCodeCompiler.scala
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.runtime2
+
+import java.io.ByteArrayInputStream
+
+import org.apache.daffodil.compiler.Compiler
+import org.apache.daffodil.infoset.DIComplex
+import org.apache.daffodil.infoset.TestInfoset
+import org.apache.daffodil.util.Misc
+import org.apache.daffodil.util.SchemaUtils
+import org.junit.Test
+
+// Now that we can run TDML tests with runtime2, this test's remaining
+// value is for debugging of runtime2 components.
+class TestGeneratedCodeCompiler {
+
+  @Test
+  def compileRunParseInt32(): Unit = {
+    // Compile a DFDL schema to parse int32 numbers
+    val testSchema = SchemaUtils.dfdlTestSchema(
+        <xs:include schemaLocation="org/apache/daffodil/xsd/DFDLGeneralFormat.dfdl.xsd"/>,
+        <dfdl:format representation="binary" ref="GeneralFormat"/>,
+      <xs:element name="c1">
+        <xs:complexType>
+          <xs:sequence>
+            <xs:element name="e1" type="xs:int"/>
+            <xs:element name="c2">
+              <xs:complexType>
+                <xs:sequence>
+                  <xs:element name="e2" type="xs:int"/>
+                  <xs:element name="e3" type="xs:int"/>
+                </xs:sequence>
+              </xs:complexType>
+            </xs:element>
+          </xs:sequence>
+        </xs:complexType>
+      </xs:element>)
+    val schemaCompiler = Compiler()
+    val pf = schemaCompiler.compileNode(testSchema)
+    assert(!pf.isError, pf.getDiagnostics.map(_.getMessage()).mkString("\n"))
+    // Generate C code from the DFDL schema
+    val codeGeneratorState = pf.generateCode()
+    val generatedCodeCompiler = new GeneratedCodeCompiler(pf)
+    val rootElementName = "c1"
+    generatedCodeCompiler.compile(rootElementName, codeGeneratorState)
+    assert(!pf.isError, pf.getDiagnostics.map(_.getMessage()).mkString("\n"))
+    // Run the executable to parse int32 numbers
+    val dp = generatedCodeCompiler.dataProcessor
+    val b = Misc.hex2Bytes("000000010000000200000003")
+    val input = new ByteArrayInputStream(b)
+    val pr = dp.parse(input)
+    assert(!pr.isError && pf.getDiagnostics.isEmpty, pr.getDiagnostics.map(_.getMessage()).mkString("\n"))
+    // Create an internal Daffodil infoset from the XML file
+    val (infoset: DIComplex, _, tunables) = TestInfoset.testInfoset(testSchema, pr.infosetAsXML)
+    assert(infoset.hasVisibleChildren)
+    assert(infoset.erd.name == "c1")

Review comment:
       We should explore the infoset more here, and insure it matches the schema. In principle we could validate the data against the schema rather than do some ad-hoc explore of it. 

##########
File path: daffodil-runtime2/src/main/c/common_runtime.c
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.
+ */
+
+#include "common_runtime.h"
+
+// walkInfosetNode - recursively walk an infoset node and call
+// VisitEventHandler methods
+
+static const char *
+walkInfosetNode(const VisitEventHandler *handler, const InfosetBase *infoNode)

Review comment:
       Is there some convention for function comments that enables javadoc-like tools to create HTML doc out of them?

##########
File path: daffodil-runtime2/src/main/c/daffodil_argp.h
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.
+ */
+
+#ifndef DAFFODIL_ARGP_H
+#define DAFFODIL_ARGP_H
+
+// Parse our "daffodil" command line interface
+
+extern int parse_daffodil_cli(int argc, char **argv);
+
+// Get our "daffodil" CLI options
+
+extern struct daffodil_cli
+{
+    enum daffodil_subcommand
+    {
+        DAFFODIL_NONE,
+        DAFFODIL_PARSE,
+        DAFFODIL_UNPARSE
+    } subcommand;
+    int verbosity;
+} daffodil_cli;
+
+// Get our "daffodil parse" CLI options
+
+extern struct daffodil_parse_cli
+{
+    const char *infoset_type;

Review comment:
       can we call this the infoset_converter? or external_infoset_rep? 
   
   The C infoset is the structs that extend InfosetBase. We convert that infoset to xml (or eventually other things if we want) for the CLI to make the data tangible for users learning DFDL, and TDML test-runner reasons.
   
   This is again about trying to keep separation of all things that convert the C infoset to/from JSON, XML, etc. being separate from the "pure" runtime which should have no code specific to XML, JSON, etc. 

##########
File path: daffodil-runtime2/src/main/c/generated_code.h
##########
@@ -0,0 +1,44 @@
+/*

Review comment:
       Also belongs in daffodil-runtime2/src/test/c/handCraftedExample1.h

##########
File path: daffodil-runtime2/src/main/c/daffodil_main.c
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.
+ */
+
+#include "common_runtime.h" // for walkInfoset, InfosetBase, ERD, ...
+#include "daffodil_argp.h"  // for daffodil_cli, parse_daffodil_cli, ...
+#include "generated_code.h" // for rootInfoset

Review comment:
       Do we actually generate this? I think this should be called root.h.
   
   I am assuming the actual generated code defines the rootInfoset() function, so that function has a prototype here, but no implementation. 
   

##########
File path: daffodil-runtime2/src/main/c/xml_reader.c
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.
+ */
+
+#include "xml_reader.h"
+#include <errno.h>  // for errno, ERANGE
+#include <limits.h> // for LONG_MAX, LONG_MIN
+#include <mxml.h>   // for mxmlWalkNext, mxmlGetElement, mxmlGetType, ...
+#include <stdint.h> // for int32_t, INT32_MAX, INT32_MIN
+#include <stdlib.h> // for NULL, strtol
+#include <string.h> // for strcmp, strerror
+
+// Read XML data from file before walking infoset
+
+static const char *
+xmlStartDocument(XMLReader *reader)
+{
+    // Load the XML data into memory
+    reader->xml = mxmlLoadFile(NULL, reader->stream, MXML_OPAQUE_CALLBACK);
+    reader->node = reader->xml;
+    if (reader->node == NULL) {
+        return "Unable to read XML data from input file";
+    }
+
+    // Consume the <?xml line if there is one
+    const char *name = mxmlGetElement(reader->node);
+    if (name && strncmp(name, "?xml", strlen("?xml")) == 0) {
+        do
+        {
+            reader->node = mxmlWalkNext(reader->node, reader->xml, MXML_DESCEND);
+        } while (mxmlGetType(reader->node) == MXML_OPAQUE);
+        name = mxmlGetElement(reader->node);
+    }
+
+    // Consume a comment if there is one
+    if (name && strncmp(name, "!--", strlen("!--")) == 0) {
+        do
+        {
+            reader->node = mxmlWalkNext(reader->node, reader->xml, MXML_DESCEND);
+        } while (mxmlGetType(reader->node) == MXML_OPAQUE);
+    }
+
+    return reader->node ? NULL : "Ran out of XML data";
+}
+
+// Delete XML data after walking infoset
+
+static const char *
+xmlEndDocument(XMLReader *reader)
+{
+    // Consume any remaining newlines or whitespace
+    while (mxmlGetType(reader->node) == MXML_OPAQUE) {
+        reader->node = mxmlWalkNext(reader->node, reader->xml, MXML_DESCEND);
+    }
+
+    // Check whether we have consumed all of the XML data
+    if (reader->node != NULL) {
+        // This code path exits the program - no need to call mxmlDelete
+        return "Did not consume all of the XML data";
+    }
+
+    // Free the storage allocated to hold the XML data
+    mxmlDelete(reader->xml);
+    reader->xml = NULL;
+    reader->node = NULL;
+    return NULL;
+}
+
+// Continue walking both XML data and infoset in lockstep
+
+static const char *
+xmlStartComplex(XMLReader *reader, const InfosetBase *base)
+{
+    // Consume any newlines or whitespace before the element
+    while (mxmlGetType(reader->node) == MXML_OPAQUE) {
+        reader->node = mxmlWalkNext(reader->node, reader->xml, MXML_DESCEND);
+    }
+
+    // Get the element and consume it
+    const char *name_from_xml = mxmlGetElement(reader->node);
+    const char *name_from_infoset = base->erd->namedQName.name;
+    reader->node = mxmlWalkNext(reader->node, reader->xml, MXML_DESCEND);
+
+    // Check whether we are walking both XML data and infoset in lockstep
+    if (name_from_xml && name_from_infoset)

Review comment:
       Shouldn't this test be before we call mxmlWalkNext(...) ?

##########
File path: daffodil-runtime2/src/main/c/daffodil_argp.c
##########
@@ -0,0 +1,302 @@
+/*
+ * 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.
+ */
+
+#include "daffodil_argp.h" // for daffodil_cli, daffodil_parse_cli, ...
+#include <argp.h>          // for argp_state, argp_error, error_t, argp_parse
+#include <stdio.h>         // for sprintf
+#include <stdlib.h>        // for putenv, NULL
+#include <string.h>        // for strlen, strcmp
+
+// Initialize our "daffodil" name and version
+
+const char *argp_program_version = "Apache Daffodil (runtime2) 0.1";
+
+// Initialize our "daffodil parse" CLI options
+
+struct daffodil_parse_cli daffodil_parse = {
+    "xml", // default infoset type
+    "-",   // default infile
+    "-",   // default outfile
+};
+
+static const struct argp_option parse_options[] = {
+    {"infoset-type", 'I', "<infoset_type>", 0,
+     "Infoset type to output. Must be one of 'xml' or 'null'"},
+
+    {"output", 'o', "<file>", 0,
+     "Write output to a given file. If not given or is -, output is written to "
+     "stdout"},
+
+    {0}};
+
+static error_t parse_handler(int key, char *arg, struct argp_state *state);
+
+static const char parse_args_doc[] = "[infile]";
+
+static const char parse_doc[] =
+    "\n"
+    "Parse a file using a DFDL schema\n"
+    "\n"
+    "Parse Options:"
+    "\v"
+    " Trailing arguments:\n"
+    "  infile (not required)      input file to parse. "
+    "If not specified, or a value of -, reads from stdin";
+
+static const struct argp parse_argp = {
+    parse_options,  // array of CLI options
+    parse_handler,  // function to get these CLI options
+    parse_args_doc, // short usage documentation
+    parse_doc,      // long help documentation
+};
+
+// Handle callbacks to get our "daffodil parse" CLI options
+
+static error_t
+parse_handler(int key, char *arg, struct argp_state *state)
+{
+    struct daffodil_parse_cli *parse = state->input;
+
+    switch (key)
+    {
+    case 'I':
+        parse->infoset_type = arg;
+        break;
+
+    case 'o':
+        parse->outfile = arg;
+        break;
+
+    case ARGP_KEY_ARG:
+        if (state->arg_num)
+        {
+            argp_error(state, "too many arguments: %s", arg);
+        }
+        parse->infile = arg;
+        break;
+
+    default:
+        return ARGP_ERR_UNKNOWN;
+    }
+
+    return 0;
+}
+
+// Parse our "daffodil parse" command line interface
+
+static error_t
+parse_daffodil_parse_cli(struct argp_state *state)
+{
+    int    argc = state->argc - state->next + 1;
+    char **argv = &state->argv[state->next - 1];
+    char * old_cmd = argv[0];
+    char   new_cmd[strlen(state->name) + strlen(" parse") + 1];
+
+    sprintf(new_cmd, "%s parse", state->name);
+    argv[0] = new_cmd;
+
+    error_t status = argp_parse(&parse_argp, argc, argv, ARGP_IN_ORDER, &argc,
+                                &daffodil_parse);
+
+    argv[0] = old_cmd;
+    state->next += argc - 1;
+
+    return status;
+}
+
+// Initialize our "daffodil unparse" CLI options
+
+struct daffodil_unparse_cli daffodil_unparse = {
+    "xml", // default infoset type
+    "-",   // default infile
+    "-",   // default outfile
+};
+
+static const struct argp_option unparse_options[] = {
+    {"infoset-type", 'I', "<infoset_type>", 0,
+     "Infoset type to unparse. Must be 'xml'"},
+
+    {"output", 'o', "<file>", 0,
+     "Write output to file. If not given or is -, output is written to "
+     "standard output"},
+
+    {0}};
+
+static error_t unparse_handler(int key, char *arg, struct argp_state *state);
+
+static const char unparse_args_doc[] = "[infile]";
+
+static const char unparse_doc[] =
+    "\n"
+    "Unparse an infoset file using a DFDL schema\n"
+    "\n"
+    "Unparse Options:"
+    "\v"
+    " Trailing arguments:\n"
+    "  infile (not required)      input file to unparse. If not specified, or "
+    "a value of -, reads from stdin";
+
+static const struct argp unparse_argp = {
+    unparse_options,  // array of CLI options
+    unparse_handler,  // function to get these CLI options
+    unparse_args_doc, // short usage documentation
+    unparse_doc,      // long help documentation
+};
+
+// Handle callbacks to get our "daffodil unparse" CLI options
+
+static error_t
+unparse_handler(int key, char *arg, struct argp_state *state)
+{
+    struct daffodil_unparse_cli *unparse = state->input;
+
+    switch (key)
+    {
+    case 'I':
+        unparse->infoset_type = arg;
+        break;
+
+    case 'o':
+        unparse->outfile = arg;
+        break;
+
+    case ARGP_KEY_ARG:
+        if (state->arg_num)
+        {
+            argp_error(state, "too many arguments: %s", arg);
+        }
+        unparse->infile = arg;
+        break;
+
+    default:
+        return ARGP_ERR_UNKNOWN;
+    }
+
+    return 0;
+}
+
+// Parse our "daffodil unparse" command line interface
+
+static error_t
+parse_daffodil_unparse_cli(struct argp_state *state)
+{
+    int    argc = state->argc - state->next + 1;
+    char **argv = &state->argv[state->next - 1];
+    char * old_cmd = argv[0];
+    char   new_cmd[strlen(state->name) + strlen(" unparse") + 1];
+
+    sprintf(new_cmd, "%s unparse", state->name);
+    argv[0] = new_cmd;
+
+    error_t status = argp_parse(&unparse_argp, argc, argv, ARGP_IN_ORDER, &argc,
+                                &daffodil_unparse);
+
+    argv[0] = old_cmd;
+    state->next += argc - 1;
+
+    return status;
+}
+
+// Initialize our "daffodil" CLI options
+
+struct daffodil_cli daffodil_cli = {
+    DAFFODIL_NONE, // default subcommand
+    0,             // default verbosity
+};
+
+static const struct argp_option daffodil_options[] = {
+    {"verbose", 'v', 0, 0, "Increment verbosity level, one level for each -v",
+     -1},
+
+    {0}};
+
+static error_t daffodil_handler(int key, char *arg, struct argp_state *state);
+
+static const char daffodil_args_doc[] = "<subcommand> [SUBCOMMAND_OPTION...]";
+
+static const char daffodil_doc[] =
+    "\n"
+    "Global Options:"
+    "\v"
+    "Subcommands:\n"
+    "  parse         Parse data to a DFDL infoset\n"
+    "  unparse       Unparse a DFDL infoset\n"
+    "\n"
+    "Run 'daffodil <subcommand> --help' for subcommand specific options";
+
+static const struct argp daffodil_argp = {
+    daffodil_options,  // array of CLI options
+    daffodil_handler,  // function to get these CLI options
+    daffodil_args_doc, // short usage documentation
+    daffodil_doc,      // long help documentation
+};
+
+// Handle callbacks to get our "daffodil" CLI options
+
+static error_t
+daffodil_handler(int key, char *arg, struct argp_state *state)
+{
+    struct daffodil_cli *daffodil = state->input;
+    error_t              status = 0;
+
+    switch (key)
+    {
+    case 'v':
+        daffodil->verbosity++;
+        break;
+
+    case ARGP_KEY_ARG:
+        if (strcmp(arg, "parse") == 0)
+        {
+            daffodil->subcommand = DAFFODIL_PARSE;
+            status = parse_daffodil_parse_cli(state);

Review comment:
       Ok, way too many overloadings of "parse" here. Let's not use the term "parse" for anything but DFDL-driven parsing. If we're parsing the arguments to the CLI, can we call that cliArgParse ?
   
   Is there a getOpt type of library for C that we can use to shorten this code? Seems like a lot of code to just implement a command line args system.  
   
   Perhaps this is just that I am not used to how verbose C is and how many lines of code it takes to implement things. 
   
   I think putting "daffodil" in these function names just makes them longer, and doesn't add clarity. Or.... is that some sort of convention that "daffodil" functions are considered the "public" ones of this code file, and the others are internal/private?
   

##########
File path: daffodil-runtime2/src/main/c/common_runtime.h
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.
+ */
+
+#ifndef COMMON_RUNTIME_H
+#define COMMON_RUNTIME_H
+
+#include <stddef.h> // for ptrdiff_t
+#include <stdint.h> // for int32_t
+#include <stdio.h>  // for FILE, size_t
+
+// Prototypes needed for compilation
+
+typedef struct ElementRuntimeData ERD;
+typedef struct InfosetBase        InfosetBase;
+typedef struct PState             PState;
+typedef struct UState             UState;
+typedef struct VisitEventHandler  VisitEventHandler;
+
+typedef void (*ERDInitSelf)(InfosetBase *infoNode);
+typedef const char *(*ERDParseSelf)(InfosetBase * infoNode,
+                                    const PState *pstate);
+typedef const char *(*ERDUnparseSelf)(const InfosetBase *infoNode,
+                                      const UState *     ustate);
+
+typedef const char *(*VisitStartDocument)(const VisitEventHandler *handler);
+typedef const char *(*VisitEndDocument)(const VisitEventHandler *handler);
+typedef const char *(*VisitStartComplex)(const VisitEventHandler *handler,
+                                         const InfosetBase *      base);
+typedef const char *(*VisitEndComplex)(const VisitEventHandler *handler,
+                                       const InfosetBase *      base);
+typedef const char *(*VisitInt32Elem)(const VisitEventHandler *handler,
+                                      const ERD *erd, const int32_t *location);
+
+// NamedQName - name of an infoset element
+
+typedef struct NamedQName
+{
+    char *name;  // element name (including prefix if any)
+    char *xmlns; // xmlns attribute name (including prefix if any)
+    char *ns;    // xmlns attribute value (a namespace URI)
+} NamedQName;
+
+// TypeCode - type of an infoset element
+
+enum TypeCode
+{
+    COMPLEX,
+    PRIMITIVE_INT32
+};
+
+// ERD - element runtime data needed to parse/unparse objects
+
+typedef struct ElementRuntimeData
+{
+    const NamedQName    namedQName;
+    const enum TypeCode typeCode;
+    const size_t        numChildren;
+    const ptrdiff_t *   offsets;
+    const ERD **        childrenERDs;
+
+    const ERDInitSelf    initSelf;
+    const ERDParseSelf   parseSelf;
+    const ERDUnparseSelf unparseSelf;

Review comment:
       Getting all this const stuff right is awesome! So these members can only be initialized, not assigned. Great. 

##########
File path: daffodil-runtime2/src/main/c/xml_reader.c
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.
+ */
+
+#include "xml_reader.h"
+#include <errno.h>  // for errno, ERANGE
+#include <limits.h> // for LONG_MAX, LONG_MIN
+#include <mxml.h>   // for mxmlWalkNext, mxmlGetElement, mxmlGetType, ...
+#include <stdint.h> // for int32_t, INT32_MAX, INT32_MIN
+#include <stdlib.h> // for NULL, strtol
+#include <string.h> // for strcmp, strerror
+
+// Read XML data from file before walking infoset
+
+static const char *
+xmlStartDocument(XMLReader *reader)
+{
+    // Load the XML data into memory
+    reader->xml = mxmlLoadFile(NULL, reader->stream, MXML_OPAQUE_CALLBACK);
+    reader->node = reader->xml;
+    if (reader->node == NULL) {
+        return "Unable to read XML data from input file";
+    }
+
+    // Consume the <?xml line if there is one
+    const char *name = mxmlGetElement(reader->node);
+    if (name && strncmp(name, "?xml", strlen("?xml")) == 0) {
+        do
+        {
+            reader->node = mxmlWalkNext(reader->node, reader->xml, MXML_DESCEND);
+        } while (mxmlGetType(reader->node) == MXML_OPAQUE);
+        name = mxmlGetElement(reader->node);
+    }
+
+    // Consume a comment if there is one
+    if (name && strncmp(name, "!--", strlen("!--")) == 0) {
+        do
+        {
+            reader->node = mxmlWalkNext(reader->node, reader->xml, MXML_DESCEND);
+        } while (mxmlGetType(reader->node) == MXML_OPAQUE);
+    }
+
+    return reader->node ? NULL : "Ran out of XML data";
+}
+
+// Delete XML data after walking infoset
+
+static const char *
+xmlEndDocument(XMLReader *reader)
+{
+    // Consume any remaining newlines or whitespace
+    while (mxmlGetType(reader->node) == MXML_OPAQUE) {
+        reader->node = mxmlWalkNext(reader->node, reader->xml, MXML_DESCEND);
+    }
+
+    // Check whether we have consumed all of the XML data
+    if (reader->node != NULL) {
+        // This code path exits the program - no need to call mxmlDelete
+        return "Did not consume all of the XML data";
+    }
+
+    // Free the storage allocated to hold the XML data
+    mxmlDelete(reader->xml);
+    reader->xml = NULL;
+    reader->node = NULL;
+    return NULL;
+}
+
+// Continue walking both XML data and infoset in lockstep
+
+static const char *
+xmlStartComplex(XMLReader *reader, const InfosetBase *base)
+{
+    // Consume any newlines or whitespace before the element
+    while (mxmlGetType(reader->node) == MXML_OPAQUE) {
+        reader->node = mxmlWalkNext(reader->node, reader->xml, MXML_DESCEND);
+    }
+
+    // Get the element and consume it
+    const char *name_from_xml = mxmlGetElement(reader->node);
+    const char *name_from_infoset = base->erd->namedQName.name;
+    reader->node = mxmlWalkNext(reader->node, reader->xml, MXML_DESCEND);
+
+    // Check whether we are walking both XML data and infoset in lockstep
+    if (name_from_xml && name_from_infoset)
+    {
+        return strcmp(name_from_xml, name_from_infoset) == 0
+                   ? NULL
+                   : "Found mismatch between XML data and infoset";
+    }
+    else
+    {
+        return "Ran out of XML data";
+    }
+}
+
+// Consume XML data only on start events, not end events
+
+static const char *
+xmlEndComplex(XMLReader *reader, const InfosetBase *base)
+{
+    (void)reader;
+    (void)base;
+    return NULL;
+}
+
+// Read 32-bit integer value from XML data
+
+static const char *
+xmlInt32Elem(XMLReader *reader, const ERD *erd, int32_t *location)
+{
+    // Consume any newlines or whitespace before the element
+    while (mxmlGetType(reader->node) == MXML_OPAQUE) {
+        reader->node = mxmlWalkNext(reader->node, reader->xml, MXML_DESCEND);
+    }
+
+    // Get the element and consume it
+    const char *name_from_xml = mxmlGetElement(reader->node);
+    const char *number_from_xml = mxmlGetOpaque(reader->node);

Review comment:
       so mxml library doesn't have  a mxmlGetInt that does the strtol itself? Based on the way you've written this, it seems that every simple type must be read as mxmlGetOpaque then converted to a value type. I would expect if they have a "Opaque" call, that they'd have complementary type-specific functions also. But maybe not.
   

##########
File path: daffodil-runtime2/src/main/c/common_runtime.h
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.
+ */
+
+#ifndef COMMON_RUNTIME_H
+#define COMMON_RUNTIME_H
+
+#include <stddef.h> // for ptrdiff_t
+#include <stdint.h> // for int32_t
+#include <stdio.h>  // for FILE, size_t
+
+// Prototypes needed for compilation
+
+typedef struct ElementRuntimeData ERD;
+typedef struct InfosetBase        InfosetBase;
+typedef struct PState             PState;
+typedef struct UState             UState;
+typedef struct VisitEventHandler  VisitEventHandler;
+
+typedef void (*ERDInitSelf)(InfosetBase *infoNode);
+typedef const char *(*ERDParseSelf)(InfosetBase * infoNode,
+                                    const PState *pstate);
+typedef const char *(*ERDUnparseSelf)(const InfosetBase *infoNode,
+                                      const UState *     ustate);
+
+typedef const char *(*VisitStartDocument)(const VisitEventHandler *handler);
+typedef const char *(*VisitEndDocument)(const VisitEventHandler *handler);
+typedef const char *(*VisitStartComplex)(const VisitEventHandler *handler,
+                                         const InfosetBase *      base);
+typedef const char *(*VisitEndComplex)(const VisitEventHandler *handler,
+                                       const InfosetBase *      base);
+typedef const char *(*VisitInt32Elem)(const VisitEventHandler *handler,
+                                      const ERD *erd, const int32_t *location);
+
+// NamedQName - name of an infoset element
+
+typedef struct NamedQName
+{
+    char *name;  // element name (including prefix if any)
+    char *xmlns; // xmlns attribute name (including prefix if any)

Review comment:
       What do we need this for? Feels very TDML runner and XML-conversion specific. You are equiping the NamedQName structure with something for synthesizing xmlns:prefix="namespaceURI" type declarations. That's only needed if you are synthesizing XML. From purely DFDL perspective, we need prefix string or null here so diagnostics can refer to the element name using the DFDL schema's terminology. 
   
   It's hard to keep the things only needed for XML/TDML separate, but this is a small thing that is an artifact of that I think. 

##########
File path: daffodil-runtime2/src/main/c/xml_writer.c
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.
+ */
+
+#include "xml_writer.h"
+#include "stack.h"  // for stack_is_empty, stack_pop, stack_push, stack_top
+#include <assert.h> // for assert
+#include <mxml.h>   // for mxml_node_t, mxmlNewElement, mxmlNewOpaquef, ...
+#include <stdint.h> // for int32_t
+#include <stdio.h>  // for NULL, fflush
+
+// Push new XML document on stack.  This function is not
+// thread-safe since it uses static storage.
+
+static const char *
+xmlStartDocument(XMLWriter *writer)
+{
+#define MAX_DEPTH 100
+    static mxml_node_t *array[MAX_DEPTH];
+    stack_init(&writer->stack, array, MAX_DEPTH);
+
+    mxml_node_t *xml = mxmlNewXML("1.0");
+    stack_push(&writer->stack, xml);
+    return xml != NULL ? NULL : "Error making new XML declaration";
+}
+
+// Pop completed XML document off stack and write it to stream
+
+static const char *
+xmlEndDocument(XMLWriter *writer)
+{
+    mxml_node_t *xml = stack_pop(&writer->stack);
+    assert(stack_is_empty(&writer->stack));
+    int status = mxmlSaveFile(xml, writer->stream, MXML_NO_CALLBACK);
+    if (status < 0)
+    {
+        return "Error writing XML document";
+    }
+    status = fflush(writer->stream);
+    mxmlDelete(xml);
+    return status == 0 ? NULL : "Error flushing stream";
+}
+
+// Push new complex element on stack
+
+static const char *
+xmlStartComplex(XMLWriter *writer, const InfosetBase *base)
+{
+    mxml_node_t *complex = NULL;
+    if (!stack_is_full(&writer->stack))
+    {
+        mxml_node_t *parent = stack_top(&writer->stack);
+        char *       name = base->erd->namedQName.name;
+        char *       xmlns = base->erd->namedQName.xmlns;

Review comment:
       So this is the string that will be "xmlns" if the namespace is the default namespace. Yeah, it would be better if the data structures of the InfoBase objects didn't have artifacts of mxml's api in them. 

##########
File path: daffodil-runtime2/src/main/c/generated_code.c
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.
+ */
+
+#include "generated_code.h"

Review comment:
       So why is this file in the code base? It's an example of what we want generated code to look like right?
   I think this belongs in daffodil-runtime2/src/test/c/handCraftedExample1.c So that it is clear it is test-related code, not part of the runtime2 library, nor something that is generated. 

##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/runtime2/generators/ParserGenerator.scala
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.runtime2.generators
+
+import org.apache.daffodil.api.DFDL
+import org.apache.daffodil.dpath.NodeInfo
+import org.apache.daffodil.dpath.NodeInfo.PrimType
+import org.apache.daffodil.dsom.ElementBase
+import org.apache.daffodil.exceptions.ThrowsSDE
+
+import scala.collection.mutable
+
+/**
+ * Gives an object the ability to generate code.
+ */
+trait ParserGenerator {
+  def generateCode(state: CodeGeneratorState): Unit
+}
+
+/**
+ * Builds up the state of generated code.
+ */
+class CodeGeneratorState extends DFDL.CodeGeneratorState {
+  private val structs = mutable.Stack[ComplexCGState]()
+  private val prototypes = mutable.ArrayBuffer[String]()
+  private val erds = mutable.ArrayBuffer[String]()
+  private val finalStructs = mutable.ArrayBuffer[String]()
+  private val finalImplementation = mutable.ArrayBuffer[String]()
+
+  def addImplementation(context: ElementBase): Unit = {
+    val C = context.namedQName.local
+    val initStatements = structs.top.initStatements.mkString("\n")
+    val parserStatements = structs.top.parserStatements.mkString("\n")
+    val unparserStatements = structs.top.unparserStatements.mkString("\n")
+    val prototypeFunctions =
+      s"""static void        ${C}_initSelf($C *instance);
+         |static const char *${C}_parseSelf($C *instance, const PState *pstate);
+         |static const char *${C}_unparseSelf(const $C *instance, const UState *ustate);""".stripMargin
+    prototypes += prototypeFunctions
+    val functions =
+      s"""static void
+         |${C}_initSelf($C *instance)
+         |{
+         |$initStatements
+         |}
+         |
+         |static const char *
+         |${C}_parseSelf($C *instance, const PState *pstate)
+         |{
+         |    const char *error_msg = NULL;
+         |$parserStatements
+         |    return error_msg;
+         |}
+         |
+         |static const char *
+         |${C}_unparseSelf(const $C *instance, const UState *ustate)
+         |{
+         |    const char *error_msg = NULL;
+         |$unparserStatements
+         |    return error_msg;
+         |}
+         |""".stripMargin
+    finalImplementation += functions
+  }
+
+  private def defineQNameInit(context: ElementBase): String = {
+    val qname = context.namedQName.toQNameString
+    val xmlns = if (context.namedQName.prefix.isDefined) s"xmlns:${context.namedQName.prefix.get}" else "xmlns"
+    val ns = context.namedQName.namespace.toStringOrNullIfNoNS
+    // Optimize away xmlns=ns declaration if possible, although this approach may not be entirely correct
+    val parentOpt = context.enclosingElements.headOption
+    val parentNs = if (parentOpt.isDefined) parentOpt.get.namedQName.namespace.toStringOrNullIfNoNS
+    val qnameInit = if (ns == null || ns == parentNs)
+      s"""    {"$qname"},       // namedQName.name"""
+    else
+      s"""    {
+         |        "$qname",              // namedQName.name
+         |        "$xmlns",           // namedQName.xmlns
+         |        "$ns", // namedQName.ns
+         |    },""".stripMargin
+    qnameInit
+  }
+
+  def addComplexTypeERD(context: ElementBase): Unit = {
+    val C = context.namedQName.local
+    val count = structs.top.declarations.length
+    val offsetComputations = structs.top.offsetComputations.mkString(",\n")
+    val erdComputations = structs.top.erdComputations.mkString(",\n")
+    val qnameInit = defineQNameInit(context)
+    val complexERD =
+      s"""static const $C ${C}_compute_ERD_offsets;
+         |
+         |static const ptrdiff_t ${C}_offsets[$count] = {
+         |$offsetComputations
+         |};
+         |
+         |static const ERD *${C}_childrenERDs[$count] = {
+         |$erdComputations
+         |};
+         |
+         |static const ERD ${C}_ERD = {
+         |$qnameInit
+         |    COMPLEX,                         // typeCode
+         |    $count,                               // numChildren
+         |    ${C}_offsets,                      // offsets
+         |    ${C}_childrenERDs,                 // childrenERDs
+         |    (ERDInitSelf)&${C}_initSelf,       // initSelf
+         |    (ERDParseSelf)&${C}_parseSelf,     // parseSelf
+         |    (ERDUnparseSelf)&${C}_unparseSelf, // unparseSelf
+         |};
+         |""".stripMargin
+    erds += complexERD
+  }
+
+  def addStruct(context: ElementBase): Unit = {
+    val C = context.namedQName.local
+    val declarations = structs.top.declarations.mkString("\n")
+    val struct =
+      s"""typedef struct $C
+         |{
+         |    InfosetBase _base;
+         |$declarations
+         |} $C;
+         |""".stripMargin
+    finalStructs += struct
+    val initStatement = s"    instance->_base.erd = &${C}_ERD;"
+    structs.top.initStatements += initStatement
+  }
+
+  def addSimpleTypeStatements(initStatement: String, parseStatement: String, unparseStatement: String): Unit = {
+    structs.top.initStatements += initStatement
+    structs.top.parserStatements += parseStatement
+    structs.top.unparserStatements += unparseStatement
+  }
+
+  def addComplexTypeStatements(child: ElementBase): Unit = {
+    val C = child.namedQName.local
+    val e = child.name
+    val initStatement = s"    ${C}_initSelf(&instance->$e);"
+    val parseStatement =
+      s"""    if (error_msg == NULL)
+         |    {
+         |        error_msg = ${C}_parseSelf(&instance->$e, pstate);
+         |    }""".stripMargin
+    val unparseStatement =
+      s"""    if (error_msg == NULL)
+         |    {
+         |        error_msg = ${C}_unparseSelf(&instance->$e, ustate);
+         |    }""".stripMargin
+    structs.top.initStatements += initStatement
+    structs.top.parserStatements += parseStatement
+    structs.top.unparserStatements += unparseStatement
+  }
+
+  def pushComplexElement(context: ElementBase): Unit = {
+    val C = context.namedQName.local
+    structs.push(new ComplexCGState(C))
+  }
+
+  def popComplexElement(context: ElementBase): Unit = {
+    structs.pop()
+  }
+
+  def addSimpleTypeERD(context: ElementBase): Unit = {
+    val e = context.namedQName.local
+    val qnameInit = defineQNameInit(context)
+    val typeCode = context.optPrimType.get match {
+      case PrimType.Int => "PRIMITIVE_INT32"
+      case PrimType.String => "PRIMITIVE_STRING"
+      case p: PrimType => context.SDE("PrimType %s not supported yet.", p.toString)
+    }
+    val erd =
+      s"""static const ERD ${e}_ERD = {
+         |$qnameInit
+         |    $typeCode, // typeCode
+         |    0,               // numChildren
+         |    NULL,            // offsets
+         |    NULL,            // childrenERDs
+         |    NULL,            // initSelf
+         |    NULL,            // parseSelf
+         |    NULL,            // unparseSelf
+         |};
+         |""".stripMargin
+    erds += erd
+    addComputations(context)
+  }
+
+  def addComputations(child: ElementBase): Unit = {
+    val C = structs.top.C
+    val e = child.namedQName.local
+    val offsetComputation = s"    (char *)&${C}_compute_ERD_offsets.$e - (char *)&${C}_compute_ERD_offsets"
+    val erdComputation = s"    &${e}_ERD"
+    structs.top.offsetComputations += offsetComputation
+    structs.top.erdComputations += erdComputation
+  }
+
+  def addFieldDeclaration(context: ThrowsSDE, child: ElementBase): Unit = {
+    val definition = if (child.isSimpleType) {
+      import NodeInfo.PrimType
+      child.optPrimType.get match {
+        case PrimType.Long => "int64_t    "
+        case PrimType.Int => "int32_t    "
+        case x => context.SDE("Unsupported primitive type: " + x)
+      }
+    } else {
+      child.namedQName.local + "         "
+    }
+    structs.top.declarations += s"    $definition ${child.name};"
+  }
+
+  def viewCodeHeader: String = {
+    val structs = finalStructs.mkString("\n")
+    val header =
+      s"""#ifndef GENERATED_CODE_H
+         |#define GENERATED_CODE_H
+         |
+         |#include "common_runtime.h" // for InfosetBase
+         |#include <stdint.h>         // for int32_t
+         |
+         |// Return the root of an infoset to be used for parsing or unparsing
+         |
+         |extern InfosetBase *rootInfoset();
+         |
+         |// Define some infoset structures
+         |
+         |$structs
+         |#endif // GENERATED_CODE_H
+         |""".stripMargin
+    header
+  }
+
+  def viewCodeFile(rootElementName: String): String = {
+    val prototypes = this.prototypes.mkString("\n")
+    val erds = this.erds.mkString("\n")
+    val finalImplementation = this.finalImplementation.mkString("\n")
+    val code =
+      s"""#include "generated_code.h"
+         |#include <endian.h> // for be32toh, htobe32
+         |#include <errno.h>  // for errno

Review comment:
       I hope we can get rid of errno. Comment below about switching to strtonum() from strtol() so as to eliminate use of errno.

##########
File path: daffodil-runtime2/src/main/c/xml_writer.c
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.
+ */
+
+#include "xml_writer.h"
+#include "stack.h"  // for stack_is_empty, stack_pop, stack_push, stack_top
+#include <assert.h> // for assert
+#include <mxml.h>   // for mxml_node_t, mxmlNewElement, mxmlNewOpaquef, ...
+#include <stdint.h> // for int32_t
+#include <stdio.h>  // for NULL, fflush
+
+// Push new XML document on stack.  This function is not
+// thread-safe since it uses static storage.
+
+static const char *
+xmlStartDocument(XMLWriter *writer)
+{
+#define MAX_DEPTH 100
+    static mxml_node_t *array[MAX_DEPTH];
+    stack_init(&writer->stack, array, MAX_DEPTH);
+
+    mxml_node_t *xml = mxmlNewXML("1.0");
+    stack_push(&writer->stack, xml);
+    return xml != NULL ? NULL : "Error making new XML declaration";
+}
+
+// Pop completed XML document off stack and write it to stream
+
+static const char *
+xmlEndDocument(XMLWriter *writer)
+{
+    mxml_node_t *xml = stack_pop(&writer->stack);
+    assert(stack_is_empty(&writer->stack));
+    int status = mxmlSaveFile(xml, writer->stream, MXML_NO_CALLBACK);
+    if (status < 0)
+    {
+        return "Error writing XML document";
+    }
+    status = fflush(writer->stream);
+    mxmlDelete(xml);
+    return status == 0 ? NULL : "Error flushing stream";
+}
+
+// Push new complex element on stack
+
+static const char *
+xmlStartComplex(XMLWriter *writer, const InfosetBase *base)
+{
+    mxml_node_t *complex = NULL;
+    if (!stack_is_full(&writer->stack))
+    {
+        mxml_node_t *parent = stack_top(&writer->stack);
+        char *       name = base->erd->namedQName.name;
+        char *       xmlns = base->erd->namedQName.xmlns;
+        char *       ns = base->erd->namedQName.ns;
+        complex = mxmlNewElement(parent, name);
+        if (xmlns != NULL)
+        {
+            mxmlElementSetAttr(complex, xmlns, ns);

Review comment:
       mxml actually considers xmlns prefix bindings to be XML attributes. I find that surprising. They're really not attributes per se. 

##########
File path: daffodil-runtime2/src/main/c/xml_reader.c
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.
+ */
+
+#include "xml_reader.h"
+#include <errno.h>  // for errno, ERANGE
+#include <limits.h> // for LONG_MAX, LONG_MIN
+#include <mxml.h>   // for mxmlWalkNext, mxmlGetElement, mxmlGetType, ...
+#include <stdint.h> // for int32_t, INT32_MAX, INT32_MIN
+#include <stdlib.h> // for NULL, strtol
+#include <string.h> // for strcmp, strerror
+
+// Read XML data from file before walking infoset
+
+static const char *
+xmlStartDocument(XMLReader *reader)
+{
+    // Load the XML data into memory
+    reader->xml = mxmlLoadFile(NULL, reader->stream, MXML_OPAQUE_CALLBACK);
+    reader->node = reader->xml;
+    if (reader->node == NULL) {
+        return "Unable to read XML data from input file";
+    }
+
+    // Consume the <?xml line if there is one
+    const char *name = mxmlGetElement(reader->node);
+    if (name && strncmp(name, "?xml", strlen("?xml")) == 0) {
+        do
+        {
+            reader->node = mxmlWalkNext(reader->node, reader->xml, MXML_DESCEND);
+        } while (mxmlGetType(reader->node) == MXML_OPAQUE);
+        name = mxmlGetElement(reader->node);
+    }
+
+    // Consume a comment if there is one
+    if (name && strncmp(name, "!--", strlen("!--")) == 0) {
+        do
+        {
+            reader->node = mxmlWalkNext(reader->node, reader->xml, MXML_DESCEND);
+        } while (mxmlGetType(reader->node) == MXML_OPAQUE);
+    }
+
+    return reader->node ? NULL : "Ran out of XML data";
+}
+
+// Delete XML data after walking infoset
+
+static const char *
+xmlEndDocument(XMLReader *reader)
+{
+    // Consume any remaining newlines or whitespace
+    while (mxmlGetType(reader->node) == MXML_OPAQUE) {
+        reader->node = mxmlWalkNext(reader->node, reader->xml, MXML_DESCEND);
+    }
+
+    // Check whether we have consumed all of the XML data
+    if (reader->node != NULL) {
+        // This code path exits the program - no need to call mxmlDelete
+        return "Did not consume all of the XML data";
+    }
+
+    // Free the storage allocated to hold the XML data
+    mxmlDelete(reader->xml);
+    reader->xml = NULL;
+    reader->node = NULL;
+    return NULL;
+}
+
+// Continue walking both XML data and infoset in lockstep
+
+static const char *
+xmlStartComplex(XMLReader *reader, const InfosetBase *base)
+{
+    // Consume any newlines or whitespace before the element
+    while (mxmlGetType(reader->node) == MXML_OPAQUE) {
+        reader->node = mxmlWalkNext(reader->node, reader->xml, MXML_DESCEND);
+    }
+
+    // Get the element and consume it
+    const char *name_from_xml = mxmlGetElement(reader->node);
+    const char *name_from_infoset = base->erd->namedQName.name;
+    reader->node = mxmlWalkNext(reader->node, reader->xml, MXML_DESCEND);
+
+    // Check whether we are walking both XML data and infoset in lockstep
+    if (name_from_xml && name_from_infoset)
+    {
+        return strcmp(name_from_xml, name_from_infoset) == 0
+                   ? NULL
+                   : "Found mismatch between XML data and infoset";
+    }
+    else
+    {
+        return "Ran out of XML data";
+    }
+}
+
+// Consume XML data only on start events, not end events
+
+static const char *
+xmlEndComplex(XMLReader *reader, const InfosetBase *base)
+{
+    (void)reader;
+    (void)base;
+    return NULL;
+}
+
+// Read 32-bit integer value from XML data
+
+static const char *
+xmlInt32Elem(XMLReader *reader, const ERD *erd, int32_t *location)
+{
+    // Consume any newlines or whitespace before the element

Review comment:
       Hmmm. NOt sure you want to do this. This makes these things equivalent:
   ```
   <foo>43</foo>
   <foo>
      43
   </foo>
   ```
   From straight XML comparison perspective those do not have the same XML Infoset. They're equivalent only if ignoring whitespace which is an application decision. The slippery slope of XML equivalency says these are also equivalent 
   ```
   <foo>abc def</foo>
   <foo>abc            def</foo>
   ```
   because adjacent spaces are collapsed. This is all XML "markup language" stuff. When using XML as a data language we have to be more careful about whitespace.
   
   Do we have to handle elements with CDATA bracketing specifically with this mxml library?
   E.g., 
   ```
   <foo><![CDATA[43]]></foo>
   ```
   Or does that get handled by the library? What about entities like "&#x20;" Does mxml convert those, or hand them through as entities for the application to deal with?
   
    

##########
File path: daffodil-tdml-processor/src/main/scala/org/apache/daffodil/tdml/processor/runtime2/Runtime2TDMLDFDLProcessor.scala
##########
@@ -0,0 +1,234 @@
+/*
+ * 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.tdml.processor.runtime2
+
+import org.apache.daffodil.api._
+import org.apache.daffodil.compiler.Compiler
+import org.apache.daffodil.externalvars.Binding
+import org.apache.daffodil.runtime2.GeneratedCodeCompiler
+import org.apache.daffodil.runtime2.ParseResult
+import org.apache.daffodil.runtime2.UnparseResult
+import org.apache.daffodil.tdml.processor._
+import org.apache.daffodil.xml.XMLUtils
+
+import scala.xml.Node
+
+final class TDMLDFDLProcessorFactory private(
+  private var compiler: Compiler,
+  private var checkAllTopLevel: Boolean,
+  validateDFDLSchemasArg: Boolean)
+  extends AbstractTDMLDFDLProcessorFactory {
+
+  override def validateDFDLSchemas = validateDFDLSchemasArg
+
+  override type R = TDMLDFDLProcessorFactory
+
+  override def implementationName = "daffodil-runtime2"
+
+  def this() = this(compiler = Compiler(validateDFDLSchemas = true),
+    checkAllTopLevel = false,
+    validateDFDLSchemasArg = true)
+
+  private def copy(
+    compiler: Compiler = compiler,
+    checkAllTopLevel: Boolean = checkAllTopLevel,
+    validateDFDLSchemas: Boolean = validateDFDLSchemas) =
+    new TDMLDFDLProcessorFactory(compiler, checkAllTopLevel, validateDFDLSchemas)
+
+  /**
+   * Deprecated methods must be implemented. Some are just stubs though now.
+   */
+  @deprecated("Use withValidateDFDLSchemas.", "2.6.0")
+  override def setValidateDFDLSchemas(bool: Boolean): Unit = {
+    compiler = compiler.withValidateDFDLSchemas(bool)
+  }
+
+  override def withValidateDFDLSchemas(bool: Boolean): TDMLDFDLProcessorFactory = {
+    copy(compiler = compiler.withValidateDFDLSchemas(bool))
+  }
+
+  @deprecated("Use withCheckAllTopLevel.", "2.6.0")
+  override def setCheckAllTopLevel(checkAllTopLevel: Boolean): Unit = {
+    compiler = compiler.withCheckAllTopLevel(checkAllTopLevel)
+  }
+
+  override def withCheckAllTopLevel(checkAllTopLevel: Boolean): TDMLDFDLProcessorFactory = {
+    copy(compiler = compiler.withCheckAllTopLevel(checkAllTopLevel))
+  }
+
+  @deprecated("Use withTunables.", "2.6.0")
+  override def setTunables(tunables: Map[String, String]): Unit =
+    compiler = compiler.withTunables(tunables)
+
+  override def withTunables(tunables: Map[String, String]): TDMLDFDLProcessorFactory =
+    copy(compiler = compiler.withTunables(tunables))
+
+  @deprecated("Use DaffodilTDMLDFDLProcessor.setExternalDFDLVariables.", "2.6.0")
+  override def setExternalDFDLVariables(externalVarBindings: Seq[Binding]): Unit =
+    compiler = compiler.withExternalDFDLVariablesImpl(externalVarBindings)
+
+  override def withExternalDFDLVariables(externalVarBindings: Seq[Binding]): TDMLDFDLProcessorFactory =
+    copy(compiler = compiler.withExternalDFDLVariablesImpl(externalVarBindings))
+
+  @deprecated("Use arguments to getProcessor()", "2.6.0")
+  override def setDistinguishedRootNode(name: String, namespace: String): Unit =
+    compiler = compiler.withDistinguishedRootNode(name, namespace)
+
+  // We're doing to replace this method with different code.
+  // Return result is a TDML.CompileResult - so it's the result
+  // of compiling the schema for the test.
+  override def getProcessor(
+    schemaSource: DaffodilSchemaSource,
+    useSerializedProcessor: Boolean,
+    optRootName: Option[String] = None,
+    optRootNamespace: Option[String] = None): TDML.CompileResult = {
+    val pf = compiler.compileSource(schemaSource, optRootName, optRootNamespace)
+    val res = if (pf.isError) {
+      Left(pf.getDiagnostics) // DFDL schema compilation diagnostics
+    } else {
+      // How can we move some of these calls to ProcessorFactory with tunable runtime = "runtime2"?
+      val rootElementName = optRootName.getOrElse("FIXME")

Review comment:
       Add a ccomment // FIXME to this line, so that tools that look for those FIXME comments find this. 

##########
File path: daffodil-tdml-lib/src/main/scala/org/apache/daffodil/tdml/TDMLRunner.scala
##########
@@ -1307,7 +1312,7 @@ case class UnparserTestCase(ptc: NodeSeq, parentArg: DFDLTestSuite)
     if (testDataLength >= 0) {
       val fullBytesNeeded = (testDataLength + 7) / 8
       if (testData.length != fullBytesNeeded) {
-        throw TDMLException("Unparse result data was was %d bytes, but the result length (%d bits) requires %d bytes.".format(
+        throw TDMLException("Unparse result data was %d bytes, but the result length (%d bits) requires %d bytes.".format(

Review comment:
       This is a bug fix in daffodil!

##########
File path: daffodil-runtime2/src/main/c/xml_reader.c
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.
+ */
+
+#include "xml_reader.h"
+#include <errno.h>  // for errno, ERANGE
+#include <limits.h> // for LONG_MAX, LONG_MIN
+#include <mxml.h>   // for mxmlWalkNext, mxmlGetElement, mxmlGetType, ...
+#include <stdint.h> // for int32_t, INT32_MAX, INT32_MIN
+#include <stdlib.h> // for NULL, strtol
+#include <string.h> // for strcmp, strerror
+
+// Read XML data from file before walking infoset
+
+static const char *
+xmlStartDocument(XMLReader *reader)
+{
+    // Load the XML data into memory
+    reader->xml = mxmlLoadFile(NULL, reader->stream, MXML_OPAQUE_CALLBACK);
+    reader->node = reader->xml;
+    if (reader->node == NULL) {
+        return "Unable to read XML data from input file";
+    }
+
+    // Consume the <?xml line if there is one
+    const char *name = mxmlGetElement(reader->node);
+    if (name && strncmp(name, "?xml", strlen("?xml")) == 0) {
+        do
+        {
+            reader->node = mxmlWalkNext(reader->node, reader->xml, MXML_DESCEND);
+        } while (mxmlGetType(reader->node) == MXML_OPAQUE);
+        name = mxmlGetElement(reader->node);
+    }
+
+    // Consume a comment if there is one
+    if (name && strncmp(name, "!--", strlen("!--")) == 0) {
+        do
+        {
+            reader->node = mxmlWalkNext(reader->node, reader->xml, MXML_DESCEND);
+        } while (mxmlGetType(reader->node) == MXML_OPAQUE);
+    }
+
+    return reader->node ? NULL : "Ran out of XML data";
+}
+
+// Delete XML data after walking infoset
+
+static const char *
+xmlEndDocument(XMLReader *reader)
+{
+    // Consume any remaining newlines or whitespace
+    while (mxmlGetType(reader->node) == MXML_OPAQUE) {
+        reader->node = mxmlWalkNext(reader->node, reader->xml, MXML_DESCEND);
+    }
+
+    // Check whether we have consumed all of the XML data
+    if (reader->node != NULL) {
+        // This code path exits the program - no need to call mxmlDelete
+        return "Did not consume all of the XML data";
+    }
+
+    // Free the storage allocated to hold the XML data
+    mxmlDelete(reader->xml);
+    reader->xml = NULL;
+    reader->node = NULL;
+    return NULL;
+}
+
+// Continue walking both XML data and infoset in lockstep
+
+static const char *
+xmlStartComplex(XMLReader *reader, const InfosetBase *base)
+{
+    // Consume any newlines or whitespace before the element
+    while (mxmlGetType(reader->node) == MXML_OPAQUE) {
+        reader->node = mxmlWalkNext(reader->node, reader->xml, MXML_DESCEND);
+    }
+
+    // Get the element and consume it
+    const char *name_from_xml = mxmlGetElement(reader->node);
+    const char *name_from_infoset = base->erd->namedQName.name;
+    reader->node = mxmlWalkNext(reader->node, reader->xml, MXML_DESCEND);
+
+    // Check whether we are walking both XML data and infoset in lockstep
+    if (name_from_xml && name_from_infoset)
+    {
+        return strcmp(name_from_xml, name_from_infoset) == 0
+                   ? NULL
+                   : "Found mismatch between XML data and infoset";
+    }
+    else
+    {
+        return "Ran out of XML data";
+    }
+}
+
+// Consume XML data only on start events, not end events
+
+static const char *
+xmlEndComplex(XMLReader *reader, const InfosetBase *base)
+{
+    (void)reader;
+    (void)base;
+    return NULL;
+}
+
+// Read 32-bit integer value from XML data
+
+static const char *
+xmlInt32Elem(XMLReader *reader, const ERD *erd, int32_t *location)
+{
+    // Consume any newlines or whitespace before the element
+    while (mxmlGetType(reader->node) == MXML_OPAQUE) {
+        reader->node = mxmlWalkNext(reader->node, reader->xml, MXML_DESCEND);
+    }
+
+    // Get the element and consume it
+    const char *name_from_xml = mxmlGetElement(reader->node);
+    const char *number_from_xml = mxmlGetOpaque(reader->node);
+    const char *name_from_infoset = erd->namedQName.name;
+    reader->node = mxmlWalkNext(reader->node, reader->xml, MXML_DESCEND);
+
+    // Check whether we are walking both XML data and infoset in lockstep
+    if (name_from_xml && name_from_infoset)
+    {
+        if (strcmp(name_from_xml, name_from_infoset) == 0)
+        {
+            // Check for errors when reading the 32-bit integer
+            char *      endptr = NULL;
+            errno = 0; // To distinguish success/failure after call

Review comment:
       Avoid errno. Pretends to be thread safe, but really isn't due to interrupt contexts making calls that use it,  and requires an extra call to get the thread-specific errno location. Slow, and causes subtle errors.
   
   Use strtonum() instead. It doesn't use errno and is a way better choice for that reason. Also does the bounds checking as part of it. It's just generally better.
   
   I realize this is the XML converter, so isn't likely to run into any errno issues. Still, strtonum() is just better anyway as it is cleaner API. 
   
   I squirm any time I see errno. 
   

##########
File path: daffodil-core/src/main/scala/org/apache/daffodil/runtime2/generators/ParserGenerator.scala
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.runtime2.generators
+
+import org.apache.daffodil.api.DFDL
+import org.apache.daffodil.dpath.NodeInfo
+import org.apache.daffodil.dpath.NodeInfo.PrimType
+import org.apache.daffodil.dsom.ElementBase
+import org.apache.daffodil.exceptions.ThrowsSDE
+
+import scala.collection.mutable
+
+/**
+ * Gives an object the ability to generate code.
+ */
+trait ParserGenerator {
+  def generateCode(state: CodeGeneratorState): Unit
+}
+
+/**
+ * Builds up the state of generated code.
+ */
+class CodeGeneratorState extends DFDL.CodeGeneratorState {
+  private val structs = mutable.Stack[ComplexCGState]()
+  private val prototypes = mutable.ArrayBuffer[String]()
+  private val erds = mutable.ArrayBuffer[String]()
+  private val finalStructs = mutable.ArrayBuffer[String]()
+  private val finalImplementation = mutable.ArrayBuffer[String]()
+
+  def addImplementation(context: ElementBase): Unit = {
+    val C = context.namedQName.local
+    val initStatements = structs.top.initStatements.mkString("\n")
+    val parserStatements = structs.top.parserStatements.mkString("\n")
+    val unparserStatements = structs.top.unparserStatements.mkString("\n")
+    val prototypeFunctions =
+      s"""static void        ${C}_initSelf($C *instance);
+         |static const char *${C}_parseSelf($C *instance, const PState *pstate);
+         |static const char *${C}_unparseSelf(const $C *instance, const UState *ustate);""".stripMargin
+    prototypes += prototypeFunctions
+    val functions =
+      s"""static void
+         |${C}_initSelf($C *instance)
+         |{
+         |$initStatements
+         |}
+         |
+         |static const char *
+         |${C}_parseSelf($C *instance, const PState *pstate)
+         |{
+         |    const char *error_msg = NULL;
+         |$parserStatements
+         |    return error_msg;
+         |}
+         |
+         |static const char *
+         |${C}_unparseSelf(const $C *instance, const UState *ustate)
+         |{
+         |    const char *error_msg = NULL;
+         |$unparserStatements
+         |    return error_msg;
+         |}
+         |""".stripMargin
+    finalImplementation += functions
+  }
+
+  private def defineQNameInit(context: ElementBase): String = {
+    val qname = context.namedQName.toQNameString
+    val xmlns = if (context.namedQName.prefix.isDefined) s"xmlns:${context.namedQName.prefix.get}" else "xmlns"
+    val ns = context.namedQName.namespace.toStringOrNullIfNoNS
+    // Optimize away xmlns=ns declaration if possible, although this approach may not be entirely correct
+    val parentOpt = context.enclosingElements.headOption
+    val parentNs = if (parentOpt.isDefined) parentOpt.get.namedQName.namespace.toStringOrNullIfNoNS
+    val qnameInit = if (ns == null || ns == parentNs)
+      s"""    {"$qname"},       // namedQName.name"""
+    else
+      s"""    {
+         |        "$qname",              // namedQName.name
+         |        "$xmlns",           // namedQName.xmlns
+         |        "$ns", // namedQName.ns
+         |    },""".stripMargin
+    qnameInit
+  }
+
+  def addComplexTypeERD(context: ElementBase): Unit = {
+    val C = context.namedQName.local
+    val count = structs.top.declarations.length
+    val offsetComputations = structs.top.offsetComputations.mkString(",\n")
+    val erdComputations = structs.top.erdComputations.mkString(",\n")
+    val qnameInit = defineQNameInit(context)
+    val complexERD =
+      s"""static const $C ${C}_compute_ERD_offsets;
+         |
+         |static const ptrdiff_t ${C}_offsets[$count] = {
+         |$offsetComputations
+         |};
+         |
+         |static const ERD *${C}_childrenERDs[$count] = {
+         |$erdComputations
+         |};
+         |
+         |static const ERD ${C}_ERD = {
+         |$qnameInit
+         |    COMPLEX,                         // typeCode
+         |    $count,                               // numChildren
+         |    ${C}_offsets,                      // offsets
+         |    ${C}_childrenERDs,                 // childrenERDs
+         |    (ERDInitSelf)&${C}_initSelf,       // initSelf
+         |    (ERDParseSelf)&${C}_parseSelf,     // parseSelf
+         |    (ERDUnparseSelf)&${C}_unparseSelf, // unparseSelf
+         |};
+         |""".stripMargin
+    erds += complexERD
+  }
+
+  def addStruct(context: ElementBase): Unit = {
+    val C = context.namedQName.local
+    val declarations = structs.top.declarations.mkString("\n")
+    val struct =
+      s"""typedef struct $C
+         |{
+         |    InfosetBase _base;
+         |$declarations
+         |} $C;
+         |""".stripMargin
+    finalStructs += struct
+    val initStatement = s"    instance->_base.erd = &${C}_ERD;"
+    structs.top.initStatements += initStatement
+  }
+
+  def addSimpleTypeStatements(initStatement: String, parseStatement: String, unparseStatement: String): Unit = {
+    structs.top.initStatements += initStatement
+    structs.top.parserStatements += parseStatement
+    structs.top.unparserStatements += unparseStatement
+  }
+
+  def addComplexTypeStatements(child: ElementBase): Unit = {
+    val C = child.namedQName.local
+    val e = child.name
+    val initStatement = s"    ${C}_initSelf(&instance->$e);"
+    val parseStatement =
+      s"""    if (error_msg == NULL)
+         |    {
+         |        error_msg = ${C}_parseSelf(&instance->$e, pstate);
+         |    }""".stripMargin
+    val unparseStatement =
+      s"""    if (error_msg == NULL)
+         |    {
+         |        error_msg = ${C}_unparseSelf(&instance->$e, ustate);
+         |    }""".stripMargin
+    structs.top.initStatements += initStatement
+    structs.top.parserStatements += parseStatement
+    structs.top.unparserStatements += unparseStatement
+  }
+
+  def pushComplexElement(context: ElementBase): Unit = {
+    val C = context.namedQName.local
+    structs.push(new ComplexCGState(C))
+  }
+
+  def popComplexElement(context: ElementBase): Unit = {
+    structs.pop()
+  }
+
+  def addSimpleTypeERD(context: ElementBase): Unit = {
+    val e = context.namedQName.local
+    val qnameInit = defineQNameInit(context)
+    val typeCode = context.optPrimType.get match {
+      case PrimType.Int => "PRIMITIVE_INT32"
+      case PrimType.String => "PRIMITIVE_STRING"
+      case p: PrimType => context.SDE("PrimType %s not supported yet.", p.toString)
+    }
+    val erd =
+      s"""static const ERD ${e}_ERD = {
+         |$qnameInit
+         |    $typeCode, // typeCode
+         |    0,               // numChildren
+         |    NULL,            // offsets
+         |    NULL,            // childrenERDs
+         |    NULL,            // initSelf
+         |    NULL,            // parseSelf
+         |    NULL,            // unparseSelf
+         |};
+         |""".stripMargin
+    erds += erd
+    addComputations(context)
+  }
+
+  def addComputations(child: ElementBase): Unit = {
+    val C = structs.top.C
+    val e = child.namedQName.local
+    val offsetComputation = s"    (char *)&${C}_compute_ERD_offsets.$e - (char *)&${C}_compute_ERD_offsets"
+    val erdComputation = s"    &${e}_ERD"
+    structs.top.offsetComputations += offsetComputation
+    structs.top.erdComputations += erdComputation
+  }
+
+  def addFieldDeclaration(context: ThrowsSDE, child: ElementBase): Unit = {
+    val definition = if (child.isSimpleType) {
+      import NodeInfo.PrimType
+      child.optPrimType.get match {
+        case PrimType.Long => "int64_t    "
+        case PrimType.Int => "int32_t    "
+        case x => context.SDE("Unsupported primitive type: " + x)
+      }
+    } else {
+      child.namedQName.local + "         "
+    }
+    structs.top.declarations += s"    $definition ${child.name};"
+  }
+
+  def viewCodeHeader: String = {
+    val structs = finalStructs.mkString("\n")
+    val header =
+      s"""#ifndef GENERATED_CODE_H
+         |#define GENERATED_CODE_H
+         |
+         |#include "common_runtime.h" // for InfosetBase
+         |#include <stdint.h>         // for int32_t
+         |
+         |// Return the root of an infoset to be used for parsing or unparsing
+         |
+         |extern InfosetBase *rootInfoset();
+         |
+         |// Define some infoset structures
+         |
+         |$structs
+         |#endif // GENERATED_CODE_H
+         |""".stripMargin
+    header
+  }
+
+  def viewCodeFile(rootElementName: String): String = {
+    val prototypes = this.prototypes.mkString("\n")
+    val erds = this.erds.mkString("\n")
+    val finalImplementation = this.finalImplementation.mkString("\n")
+    val code =
+      s"""#include "generated_code.h"
+         |#include <endian.h> // for be32toh, htobe32
+         |#include <errno.h>  // for errno
+         |#include <stddef.h> // for ptrdiff_t
+         |#include <stdio.h>  // for NULL, fread, fwrite, size_t, feof, ferror, FILE
+         |#include <string.h> // for strerror
+         |
+         |// Prototypes needed for compilation
+         |
+         |$prototypes
+         |
+         |// Metadata singletons
+         |
+         |$erds
+         |// Return the root of an infoset to be used for parsing or unparsing
+         |
+         |InfosetBase *
+         |rootInfoset()
+         |{
+         |    static $rootElementName    instance;
+         |    InfosetBase *root = &instance._base;
+         |    ${rootElementName}_ERD.initSelf(root);
+         |    return root;
+         |}
+         |
+         |// Methods to initialize, parse, and unparse infoset nodes
+         |
+         |static const char *
+         |eof_or_error_msg(FILE *stream)
+         |{
+         |    if (feof(stream))
+         |    {
+         |        static const char *error_msg = "Got EOF while expecting more input";
+         |        return error_msg;
+         |    }
+         |    else if (ferror(stream))

Review comment:
       ferror doesn't set errno. So the return strerror(errno) below ... is depending on something outside of this function having set errno appropriately. But fread/fwrite don't set errno.  Suggest we pass an arg to this specifying whether the error occurred in fread or fwrite, then we issue an error like "Error during data read for %s" where %s comes from the ERD (which we should also pass.)
   

##########
File path: daffodil-runtime2/src/main/c/generated_code.c
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.
+ */
+
+#include "generated_code.h"
+#include <endian.h> // for be32toh, htobe32
+#include <errno.h>  // for errno
+#include <stddef.h> // for ptrdiff_t
+#include <stdio.h>  // for NULL, fread, fwrite, size_t, feof, ferror, FILE
+#include <string.h> // for strerror
+
+// Prototypes needed for compilation
+
+static void        c2_initSelf(c2 *instance);
+static const char *c2_parseSelf(c2 *instance, const PState *pstate);
+static const char *c2_unparseSelf(const c2 *instance, const UState *ustate);
+static void        c1_initSelf(c1 *instance);
+static const char *c1_parseSelf(c1 *instance, const PState *pstate);
+static const char *c1_unparseSelf(const c1 *instance, const UState *ustate);
+
+// Metadata singletons
+
+static const ERD e1_ERD = {
+    {"ex:e1"},       // namedQName.name
+    PRIMITIVE_INT32, // typeCode
+    0,               // numChildren
+    NULL,            // offsets
+    NULL,            // childrenERDs
+    NULL,            // initSelf
+    NULL,            // parseSelf
+    NULL,            // unparseSelf
+};
+
+static const ERD e2_ERD = {
+    {"ex:e2"},       // namedQName.name
+    PRIMITIVE_INT32, // typeCode
+    0,               // numChildren
+    NULL,            // offsets
+    NULL,            // childrenERDs
+    NULL,            // initSelf
+    NULL,            // parseSelf
+    NULL,            // unparseSelf
+};
+
+static const ERD e3_ERD = {
+    {"ex:e3"},       // namedQName.name
+    PRIMITIVE_INT32, // typeCode
+    0,               // numChildren
+    NULL,            // offsets
+    NULL,            // childrenERDs
+    NULL,            // initSelf
+    NULL,            // parseSelf
+    NULL,            // unparseSelf
+};
+
+static const c2 c2_compute_ERD_offsets;
+
+static const ptrdiff_t c2_offsets[2] = {
+    (char *)&c2_compute_ERD_offsets.e2 - (char *)&c2_compute_ERD_offsets,
+    (char *)&c2_compute_ERD_offsets.e3 - (char *)&c2_compute_ERD_offsets};
+
+static const ERD *c2_childrenERDs[2] = {&e2_ERD, &e3_ERD};
+
+static const ERD c2_ERD = {
+    {"ex:c2"},                       // namedQName.name
+    COMPLEX,                         // typeCode
+    2,                               // numChildren
+    c2_offsets,                      // offsets
+    c2_childrenERDs,                 // childrenERDs
+    (ERDInitSelf)&c2_initSelf,       // initSelf
+    (ERDParseSelf)&c2_parseSelf,     // parseSelf
+    (ERDUnparseSelf)&c2_unparseSelf, // unparseSelf
+};
+
+static const c1 c1_compute_ERD_offsets;
+
+static const ptrdiff_t c1_offsets[2] = {
+    (char *)&c1_compute_ERD_offsets.e1 - (char *)&c1_compute_ERD_offsets,
+    (char *)&c1_compute_ERD_offsets.c2 - (char *)&c1_compute_ERD_offsets};
+
+static const ERD *c1_childrenERDs[2] = {&e1_ERD, &c2_ERD};
+
+static const ERD c1_ERD = {
+    {
+        "ex:c1",              // namedQName.name
+        "xmlns:ex",           // namedQName.xmlns
+        "http://example.com", // namedQName.ns
+    },
+    COMPLEX,                         // typeCode
+    2,                               // numChildren
+    c1_offsets,                      // offsets
+    c1_childrenERDs,                 // childrenERDs
+    (ERDInitSelf)&c1_initSelf,       // initSelf
+    (ERDParseSelf)&c1_parseSelf,     // parseSelf
+    (ERDUnparseSelf)&c1_unparseSelf, // unparseSelf
+};
+
+// Return the root of an infoset to be used for parsing or unparsing
+
+InfosetBase *
+rootInfoset()
+{
+    static c1    instance;
+    InfosetBase *root = &instance._base;
+    c1_ERD.initSelf(root);
+    return root;
+}
+
+// Methods to initialize, parse, and unparse infoset nodes
+
+static const char *
+eof_or_error_msg(FILE *stream)
+{
+    if (feof(stream))
+    {
+        static const char *error_msg = "Got EOF while expecting more input";
+        return error_msg;
+    }
+    else if (ferror(stream))
+    {
+        return strerror(errno);

Review comment:
       Note: you are depending here on fread/fwrite setting errno. They don't from the doc I quickly googled. 
   
   I find it surprising, but from my googling there is no 100% portable way to find out about the error. Only that there was one. (At least according to search I did, and stack overflow articles.)

##########
File path: daffodil-runtime2/src/main/c/daffodil_argp.c
##########
@@ -0,0 +1,302 @@
+/*
+ * 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.
+ */
+
+#include "daffodil_argp.h" // for daffodil_cli, daffodil_parse_cli, ...
+#include <argp.h>          // for argp_state, argp_error, error_t, argp_parse
+#include <stdio.h>         // for sprintf
+#include <stdlib.h>        // for putenv, NULL
+#include <string.h>        // for strlen, strcmp
+
+// Initialize our "daffodil" name and version
+
+const char *argp_program_version = "Apache Daffodil (runtime2) 0.1";
+
+// Initialize our "daffodil parse" CLI options
+
+struct daffodil_parse_cli daffodil_parse = {
+    "xml", // default infoset type
+    "-",   // default infile
+    "-",   // default outfile
+};
+
+static const struct argp_option parse_options[] = {
+    {"infoset-type", 'I', "<infoset_type>", 0,
+     "Infoset type to output. Must be one of 'xml' or 'null'"},
+
+    {"output", 'o', "<file>", 0,
+     "Write output to a given file. If not given or is -, output is written to "
+     "stdout"},
+
+    {0}};
+
+static error_t parse_handler(int key, char *arg, struct argp_state *state);
+
+static const char parse_args_doc[] = "[infile]";
+
+static const char parse_doc[] =
+    "\n"
+    "Parse a file using a DFDL schema\n"
+    "\n"
+    "Parse Options:"
+    "\v"
+    " Trailing arguments:\n"
+    "  infile (not required)      input file to parse. "
+    "If not specified, or a value of -, reads from stdin";
+
+static const struct argp parse_argp = {
+    parse_options,  // array of CLI options
+    parse_handler,  // function to get these CLI options
+    parse_args_doc, // short usage documentation
+    parse_doc,      // long help documentation
+};
+
+// Handle callbacks to get our "daffodil parse" CLI options
+
+static error_t
+parse_handler(int key, char *arg, struct argp_state *state)
+{
+    struct daffodil_parse_cli *parse = state->input;
+
+    switch (key)
+    {
+    case 'I':
+        parse->infoset_type = arg;
+        break;
+
+    case 'o':
+        parse->outfile = arg;
+        break;
+
+    case ARGP_KEY_ARG:
+        if (state->arg_num)
+        {
+            argp_error(state, "too many arguments: %s", arg);
+        }
+        parse->infile = arg;
+        break;
+
+    default:
+        return ARGP_ERR_UNKNOWN;
+    }
+
+    return 0;
+}
+
+// Parse our "daffodil parse" command line interface
+
+static error_t
+parse_daffodil_parse_cli(struct argp_state *state)
+{
+    int    argc = state->argc - state->next + 1;
+    char **argv = &state->argv[state->next - 1];
+    char * old_cmd = argv[0];
+    char   new_cmd[strlen(state->name) + strlen(" parse") + 1];
+
+    sprintf(new_cmd, "%s parse", state->name);
+    argv[0] = new_cmd;
+
+    error_t status = argp_parse(&parse_argp, argc, argv, ARGP_IN_ORDER, &argc,
+                                &daffodil_parse);
+
+    argv[0] = old_cmd;
+    state->next += argc - 1;
+
+    return status;
+}
+
+// Initialize our "daffodil unparse" CLI options
+
+struct daffodil_unparse_cli daffodil_unparse = {
+    "xml", // default infoset type
+    "-",   // default infile
+    "-",   // default outfile
+};
+
+static const struct argp_option unparse_options[] = {
+    {"infoset-type", 'I', "<infoset_type>", 0,
+     "Infoset type to unparse. Must be 'xml'"},
+
+    {"output", 'o', "<file>", 0,
+     "Write output to file. If not given or is -, output is written to "
+     "standard output"},
+
+    {0}};
+
+static error_t unparse_handler(int key, char *arg, struct argp_state *state);
+
+static const char unparse_args_doc[] = "[infile]";
+
+static const char unparse_doc[] =
+    "\n"
+    "Unparse an infoset file using a DFDL schema\n"
+    "\n"
+    "Unparse Options:"
+    "\v"
+    " Trailing arguments:\n"
+    "  infile (not required)      input file to unparse. If not specified, or "
+    "a value of -, reads from stdin";
+
+static const struct argp unparse_argp = {
+    unparse_options,  // array of CLI options
+    unparse_handler,  // function to get these CLI options
+    unparse_args_doc, // short usage documentation
+    unparse_doc,      // long help documentation
+};
+
+// Handle callbacks to get our "daffodil unparse" CLI options
+
+static error_t
+unparse_handler(int key, char *arg, struct argp_state *state)
+{
+    struct daffodil_unparse_cli *unparse = state->input;
+
+    switch (key)
+    {
+    case 'I':
+        unparse->infoset_type = arg;
+        break;
+
+    case 'o':
+        unparse->outfile = arg;
+        break;
+
+    case ARGP_KEY_ARG:
+        if (state->arg_num)
+        {
+            argp_error(state, "too many arguments: %s", arg);
+        }
+        unparse->infile = arg;
+        break;
+
+    default:
+        return ARGP_ERR_UNKNOWN;
+    }
+
+    return 0;
+}
+
+// Parse our "daffodil unparse" command line interface
+
+static error_t
+parse_daffodil_unparse_cli(struct argp_state *state)
+{
+    int    argc = state->argc - state->next + 1;
+    char **argv = &state->argv[state->next - 1];
+    char * old_cmd = argv[0];
+    char   new_cmd[strlen(state->name) + strlen(" unparse") + 1];
+
+    sprintf(new_cmd, "%s unparse", state->name);
+    argv[0] = new_cmd;
+
+    error_t status = argp_parse(&unparse_argp, argc, argv, ARGP_IN_ORDER, &argc,
+                                &daffodil_unparse);
+
+    argv[0] = old_cmd;
+    state->next += argc - 1;
+
+    return status;
+}
+
+// Initialize our "daffodil" CLI options
+
+struct daffodil_cli daffodil_cli = {
+    DAFFODIL_NONE, // default subcommand
+    0,             // default verbosity
+};
+
+static const struct argp_option daffodil_options[] = {
+    {"verbose", 'v', 0, 0, "Increment verbosity level, one level for each -v",
+     -1},
+
+    {0}};
+
+static error_t daffodil_handler(int key, char *arg, struct argp_state *state);
+
+static const char daffodil_args_doc[] = "<subcommand> [SUBCOMMAND_OPTION...]";
+
+static const char daffodil_doc[] =
+    "\n"
+    "Global Options:"
+    "\v"
+    "Subcommands:\n"
+    "  parse         Parse data to a DFDL infoset\n"
+    "  unparse       Unparse a DFDL infoset\n"
+    "\n"
+    "Run 'daffodil <subcommand> --help' for subcommand specific options";
+
+static const struct argp daffodil_argp = {
+    daffodil_options,  // array of CLI options
+    daffodil_handler,  // function to get these CLI options
+    daffodil_args_doc, // short usage documentation
+    daffodil_doc,      // long help documentation
+};
+
+// Handle callbacks to get our "daffodil" CLI options
+
+static error_t
+daffodil_handler(int key, char *arg, struct argp_state *state)
+{
+    struct daffodil_cli *daffodil = state->input;

Review comment:
       can we name this local variable "cli" instead of "daffodil" ?

##########
File path: daffodil-tdml-processor/src/main/scala/org/apache/daffodil/tdml/processor/runtime2/Runtime2TDMLDFDLProcessor.scala
##########
@@ -0,0 +1,234 @@
+/*
+ * 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.tdml.processor.runtime2
+
+import org.apache.daffodil.api._
+import org.apache.daffodil.compiler.Compiler
+import org.apache.daffodil.externalvars.Binding
+import org.apache.daffodil.runtime2.GeneratedCodeCompiler
+import org.apache.daffodil.runtime2.ParseResult
+import org.apache.daffodil.runtime2.UnparseResult
+import org.apache.daffodil.tdml.processor._
+import org.apache.daffodil.xml.XMLUtils
+
+import scala.xml.Node
+
+final class TDMLDFDLProcessorFactory private(
+  private var compiler: Compiler,
+  private var checkAllTopLevel: Boolean,
+  validateDFDLSchemasArg: Boolean)
+  extends AbstractTDMLDFDLProcessorFactory {
+
+  override def validateDFDLSchemas = validateDFDLSchemasArg
+
+  override type R = TDMLDFDLProcessorFactory
+
+  override def implementationName = "daffodil-runtime2"
+
+  def this() = this(compiler = Compiler(validateDFDLSchemas = true),
+    checkAllTopLevel = false,
+    validateDFDLSchemasArg = true)
+
+  private def copy(
+    compiler: Compiler = compiler,
+    checkAllTopLevel: Boolean = checkAllTopLevel,
+    validateDFDLSchemas: Boolean = validateDFDLSchemas) =
+    new TDMLDFDLProcessorFactory(compiler, checkAllTopLevel, validateDFDLSchemas)
+
+  /**
+   * Deprecated methods must be implemented. Some are just stubs though now.
+   */
+  @deprecated("Use withValidateDFDLSchemas.", "2.6.0")
+  override def setValidateDFDLSchemas(bool: Boolean): Unit = {
+    compiler = compiler.withValidateDFDLSchemas(bool)
+  }
+
+  override def withValidateDFDLSchemas(bool: Boolean): TDMLDFDLProcessorFactory = {
+    copy(compiler = compiler.withValidateDFDLSchemas(bool))
+  }
+
+  @deprecated("Use withCheckAllTopLevel.", "2.6.0")
+  override def setCheckAllTopLevel(checkAllTopLevel: Boolean): Unit = {
+    compiler = compiler.withCheckAllTopLevel(checkAllTopLevel)
+  }
+
+  override def withCheckAllTopLevel(checkAllTopLevel: Boolean): TDMLDFDLProcessorFactory = {
+    copy(compiler = compiler.withCheckAllTopLevel(checkAllTopLevel))
+  }
+
+  @deprecated("Use withTunables.", "2.6.0")
+  override def setTunables(tunables: Map[String, String]): Unit =
+    compiler = compiler.withTunables(tunables)
+
+  override def withTunables(tunables: Map[String, String]): TDMLDFDLProcessorFactory =
+    copy(compiler = compiler.withTunables(tunables))
+
+  @deprecated("Use DaffodilTDMLDFDLProcessor.setExternalDFDLVariables.", "2.6.0")
+  override def setExternalDFDLVariables(externalVarBindings: Seq[Binding]): Unit =
+    compiler = compiler.withExternalDFDLVariablesImpl(externalVarBindings)
+
+  override def withExternalDFDLVariables(externalVarBindings: Seq[Binding]): TDMLDFDLProcessorFactory =
+    copy(compiler = compiler.withExternalDFDLVariablesImpl(externalVarBindings))
+
+  @deprecated("Use arguments to getProcessor()", "2.6.0")
+  override def setDistinguishedRootNode(name: String, namespace: String): Unit =
+    compiler = compiler.withDistinguishedRootNode(name, namespace)
+
+  // We're doing to replace this method with different code.
+  // Return result is a TDML.CompileResult - so it's the result
+  // of compiling the schema for the test.
+  override def getProcessor(
+    schemaSource: DaffodilSchemaSource,
+    useSerializedProcessor: Boolean,
+    optRootName: Option[String] = None,
+    optRootNamespace: Option[String] = None): TDML.CompileResult = {
+    val pf = compiler.compileSource(schemaSource, optRootName, optRootNamespace)
+    val res = if (pf.isError) {
+      Left(pf.getDiagnostics) // DFDL schema compilation diagnostics
+    } else {
+      // How can we move some of these calls to ProcessorFactory with tunable runtime = "runtime2"?
+      val rootElementName = optRootName.getOrElse("FIXME")
+      val codeGeneratorState = pf.generateCode()
+      val generatedCodeCompiler = new GeneratedCodeCompiler(pf)
+      generatedCodeCompiler.compile(rootElementName, codeGeneratorState)
+      val compileResult = if (pf.isError) {
+        Left(pf.getDiagnostics) // C code compilation diagnostics
+      } else {
+        val dp = new Runtime2TDMLDFDLProcessor(generatedCodeCompiler)
+        Right((pf.getDiagnostics, dp))
+      }
+      compileResult
+    }
+    res
+  }
+
+}
+
+/**
+ * Delegates all execution, error gathering, error access to the Runtime2DataProcessor object.
+ * The responsibility of this class is just for TDML matching up. That is dealing with TDML
+ * XML Infosets, feeding to the unparser, creating XML from the result created by the
+ * Runtime2DataProcessor object. All the "real work" is done by generatedCodeCompiler.dataProcessor.
+ */
+class Runtime2TDMLDFDLProcessor(generatedCodeCompiler: GeneratedCodeCompiler) extends TDMLDFDLProcessor {
+
+  override type R = Runtime2TDMLDFDLProcessor
+
+  private val dataProcessor = generatedCodeCompiler.dataProcessor
+  private var anyErrors: Boolean = false
+  private var diagnostics: Seq[Diagnostic] = Nil
+
+  @deprecated("Use withDebugging.", "2.6.0")
+  override def setDebugging(b: Boolean) = ???
+  override def withDebugging(b: Boolean): Runtime2TDMLDFDLProcessor = this
+
+  @deprecated("Use withTracing.", "2.6.0")
+  override def setTracing(bool: Boolean): Unit = ???
+  override def withTracing(bool: Boolean): Runtime2TDMLDFDLProcessor = this
+
+  @deprecated("Use withDebugger.", "2.6.0")
+  override def setDebugger(db: AnyRef): Unit = ???
+  override def withDebugger(db: AnyRef): Runtime2TDMLDFDLProcessor = this
+
+  @deprecated("Use withValidationMode.", "2.6.0")
+  override def setValidationMode(validationMode: ValidationMode.Type): Unit = ???
+  override def withValidationMode(validationMode: ValidationMode.Type): Runtime2TDMLDFDLProcessor = this
+
+  @deprecated("Use withExternalDFDLVariables.", "2.6.0")
+  override def setExternalDFDLVariables(externalVarBindings: Seq[Binding]): Unit = ???
+  override def withExternalDFDLVariables(externalVarBindings: Seq[Binding]): Runtime2TDMLDFDLProcessor = this
+
+  // Actually run the C code and save any errors to be returned here
+  override def isError: Boolean = anyErrors
+  override def getDiagnostics: Seq[Diagnostic] = diagnostics
+
+  // This part will change a lot (it will execute C code instead).
+  // Whatever the parse produces needs to be converted into XML for comparison.
+  // We'll need a way to convert, say, a C struct to XML, and XML to C struct.
+  // The C code will need a bunch of toXML methods so it can produce output
+  // for comparison.
+  override def parse(is: java.io.InputStream, lengthLimitInBits: Long): TDMLParseResult = {
+    // We will run the generated and compiled C code, collect and save any errors
+    // and diagnostics to be returned in isError and getDiagnostics, and build an
+    // infoset.  Our context here is TDML-related, so we need to move that functionality
+    // to something generic that we call from here, you're saying.  I got it.  So we

Review comment:
       Clean up conversational comments. Much of this discussion suggests uncertainty that I think is much clearer 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