You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@daffodil.apache.org by "tuxji (via GitHub)" <gi...@apache.org> on 2023/04/01 18:39:24 UTC

[GitHub] [daffodil] tuxji opened a new pull request, #998: Add left over data check to generated C parsers

tuxji opened a new pull request, #998:
URL: https://github.com/apache/daffodil/pull/998

   Fix generated C parsers not checking for left over data after parse calls and not clearing infoset between parse calls.  Rename some infoset C functions more consistently (C snake style).
   
   Also enhance C code generator to support DFDL schemas containing simple type root elements, allowing generated C parsers to parse data and infosets containing only a single element.  Add examples and new tests for simple root elements as well.
   
   Also replace any non-alphabetical / non-numerical characters with underscores when converting XML identifiers to C identifiers, allowing XML element names like "simple-boolean" to become "simple_boolean" in generated C code.
   
   DAFFODIL-2807
   
   Main.scala: Avoid second exception if TDMLRunner throws a NullPointerException and Main tries to print its message (NPEs don't have a message).
   
   testNonCompatibleImplementation.tdml: Both ibm and daffodilC can use same schema with single element now.  Merge s1 into s2 and remove s2.
   
   TestCLItdml.scala: Remove unnecessary "-iii" options from CLI test.
   
   daffodil_main.c: Call `get_infoset(CLEAR_INFOSET)` instead of `rootElement()`, call `parse_data(infoset, &pstate)` instead of `root->erd->parseSelf(root, &pstate)`, call `walk_infoset` instead of `walkInfoset`, and call `unparse_infoset(infoset, &ustate)` instead of `root->erd->unparseSelf(root, &ustate)`, showing how you can clear the infoset between `parse_data` calls if you call the C parser in a loop.
   
   xml_reader.h: Remove unused `root` field from XMLReader struct.
   
   errors.c: Add error message for new error `ERR_LEFTOVER_DATA`.
   
   errors.h: Define new error `ERR_LEFTOVER_DATA` and remove UNUSED macro (define it in infoset.h instead).
   
   infoset.c: Rename some infoset C functions more consistently (C snake style).  Define new functions `parse_data` and `unparse_infoset` and make them call `check_pstate` and `flush_ustate` so user will only need to call `get_infoset(CLEAR_INFOSET)` and `parse_data` to get correct C parser behavior.  Also rename ERD field `offsets` to `childrenOffsets`.  Remove flushUstate function (define it as flush_ustate in unparsers.c instead).
   
   infoset.h: Rename ERD field `offsets` to `childrenOffsets`.  Rename function `rootElement(void)` to `get_infoset(clear_infoset)`.  Declare new functions `parse_data` and `unparse_infoset`.  Rename walkInfoset to walk_infoset (snake style).  Define UNUSED macro here to let extras.c use it.
   
   parsers.[ch]: Define function `check_pstate` to check for leftover data (called by `parse_data`).
   
   unparsers.[ch]: Define function `flush_ustate` to flush unwritten fractional bits (called by `unparse_infoset`).
   
   bits.c: Call `flush_ustate` instead of `flushUstate`.
   
   extras.c: Implement `get_infoset` instead of `rootElement`.
   
   DaffodilCExamplesGenerator.scala: Generate a C example with a simple root element too.
   
   BinaryBooleanCodeGenerator.scala: Convert XML name to C name.
   
   BinaryValueCodeGenerator.scala: Convert XML name to C name.
   
   CodeGeneratorState.scala: Detect the case when a root element is a simple type and translate it as a hybrid complex and simple type (that is, push state for a complex element on the stack and when popping that state, generate the hybrid offset computations and ERD declarations needed to parse the simple type root element successfully).  Define new `cName` method to convert XML names to C names.  Generate C function `get_infoset(clear_infoset)` instead of `rootElement(void)` to clear infoset between parses.  Use renamed ERD field `childrenOffsets` instead of `offsets`.
   
   HexBinaryCodeGenerator.scala: Convert XML name to C name.
   
   examples/**: Regenerate C examples due to above changes and add a C example for a simple root element schema.
   
   data/simple*.dat: Add new data files to test simple root elements.
   
   infosets/simple*.dat.xml: Add simple root element infosets.
   
   simple.dfdl.xsd: Add new DFDL schema with choice of simple root elements to test (one root element for each primitive type).
   
   simple.tdml: Add test cases for all simple root elements listed in simple.dfdl.xsd.
   
   simple-errors.tdml: Add error test cases for simple root elements listed in simple.dfdl.xsd, although don't need to test every integer type.
   
   TestSimple.scala: Add unit tests for all test cases listed in simple.tdml.
   
   TestSimpleErrors.scala: Add unit tests for all test cases listed in simple-errors.tdml.


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

To unsubscribe, e-mail: commits-unsubscribe@daffodil.apache.org

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


[GitHub] [daffodil] stevedlawrence commented on a diff in pull request #998: Add left over data check to generated C parsers

Posted by "stevedlawrence (via GitHub)" <gi...@apache.org>.
stevedlawrence commented on code in PR #998:
URL: https://github.com/apache/daffodil/pull/998#discussion_r1160850856


##########
daffodil-codegen-c/src/main/resources/org/apache/daffodil/codegen/c/files/libruntime/unparsers.c:
##########
@@ -465,3 +465,30 @@ unparse_check_bounds(const char *name, size_t count, size_t minOccurs, size_t ma
         ustate->error = &error;
     }
 }
+
+// Flush the fractional byte if not done yet
+
+void
+flush_fractional_byte(UState *ustate)
+{
+    // Skip the flush if we already have an error
+    if (!ustate->error)
+    {
+        // Do we have any unwritten bits left in the fractional byte?
+        if (ustate->numUnwritBits)
+        {
+            // Fill the fractional byte
+            size_t num_bits_fill = BYTE_WIDTH - ustate->numUnwritBits;
+            ustate->unwritBits <<= num_bits_fill;

Review Comment:
   This means the byte to write is padded with zero's instead of using the `dfdl:fillByte` property. Though, this is probably fine for runtime2. In fact, I think runtime1 might even have this same bug, so at least we are consistent :wink: Just something to be aware about that I *think* this is technically the wrong behavior.



##########
daffodil-codegen-c/src/main/scala/org/apache/daffodil/codegen/c/generators/CodeGeneratorState.scala:
##########
@@ -354,19 +365,20 @@ class CodeGeneratorState(private val root: ElementBase) {
          |// Initialize, parse, and unparse nodes of the infoset
          |
          |$finalImplementation
-         |// Return a root element for parsing or unparsing the infoset
+         |// Get an infoset (optionally clearing it first) for parsing/walking
          |
          |InfosetBase *
-         |rootElement(void)
+         |get_infoset(bool clear_infoset)
          |{
-         |    static bool initialized;
-         |    static $rootName root;
-         |    if (!initialized)
+         |    static $rootName infoset;
+         |
+         |    if (clear_infoset)
          |    {
-         |        ${rootName}_initERD(&root, (InfosetBase *)&root);
-         |        initialized = true;
+         |        memset(&infoset, 0, sizeof(infoset));

Review Comment:
   Is it necessary to walk through the infoset and free any pointers that point to heap allocated memory to avoid memory leaks?
   
   It looks like maybe that's only hexBinary? That's the only place I see malloc. Are only bounded arrays supported by runtime1 an so don't need heap allocations?



##########
daffodil-codegen-c/src/main/scala/org/apache/daffodil/codegen/c/DaffodilCExamplesGenerator.scala:
##########
@@ -81,19 +81,23 @@ object DaffodilCExamplesGenerator {
     val nestedRootName = Some("NestedUnion")
     val padTestSchema = schemaDir / "padtest.dfdl.xsd"
     val padTestRootName = None
+    val simpleSchema = schemaDir / "simple.dfdl.xsd"
+    val simpleRootName = Some("simple-byte")
     val variableLenSchema = schemaDir / "variablelen.dfdl.xsd"
     val variableLenRootName = Some("expressionElement")
 
     val examplesDir = os.Path(args(0))
     val exNumsExampleDir = examplesDir / "ex_nums"
     val nestedExampleDir = examplesDir / "NestedUnion"
     val padTestExampleDir = examplesDir / "padtest"
+    val simpleExampleDir = examplesDir / "simple"
     val variableLenExampleDir = examplesDir / "variablelen"

Review Comment:
   Doesn't have to be done in this PR, but at some point we may want to turn this into an array of tuples and we can just iterate over them, maybe something like:
   ```scala
   val examples = Array(
     ("ex_nums.dfdl.xsd", None, "ex_nums"),
     ("nested.dfdl.xsd", Some("NestedUnion"), "NestedUnion")
     ...,
   )
   examples.foreach { case (schema, rootOpt, dir) =>
     updateCExample(schemaDir / schema, rootOpt, examplesDir / dir)
   }
   ```
   Then adding new examples becomes just adding a new entry in the array and things aren't spread out among different variables.
   
   We're starting to get enough examples where this could clean things up a bit.



##########
daffodil-codegen-c/src/main/scala/org/apache/daffodil/codegen/c/generators/CodeGeneratorState.scala:
##########
@@ -727,22 +778,32 @@ class CodeGeneratorState(private val root: ElementBase) {
 
   // Recursively builds a hopefully unique name using the given StringBuilder
   private def buildName(sc: SchemaComponent, sb: StringBuilder): StringBuilder = {
+    // Append schema component's name
     sc match {
       case eb: ElementBase => sb ++= eb.namedQName.local += '_'
       case gd: GlobalElementDecl => sb ++= gd.namedQName.local += '_'
       case ct: GlobalComplexTypeDef => sb ++= ct.namedQName.local += '_'
       case _ => // don't include other schema components in qualified name
     }
+    // Recursively append parent schema components' names
     sc.optLexicalParent.foreach {
       buildName(_, sb)
     }
+    // Ensure name contains only legal characters for C identifiers
+    lazy val legalChars: immutable.Set[Char] =
+      Set('_') ++ ('a' to 'z') ++ ('A' to 'Z') ++ ('0' to '9')
+    for (i <- sb.indices) {
+      if (!legalChars.contains(sb.charAt(i))) {
+        sb.setCharAt(i, '_')
+      }
+    }

Review Comment:
   This logic about names also exists the `cName` function. Can those be refactored into a common function so there's only one place that deals with creating legal names?



##########
daffodil-codegen-c/src/main/resources/org/apache/daffodil/codegen/c/files/libruntime/infoset.c:
##########
@@ -123,64 +124,55 @@ get_erd_ns(const ERD *erd)
     return erd->namedQName.ns;
 }
 
-// walkInfoset - walk each node of an infoset and call
-// VisitEventHandler methods
+// parse_data - parse an input stream into an infoset, check for
+// leftover data, and return any errors in pstate
 
-const Error *
-walkInfoset(const VisitEventHandler *handler, const InfosetBase *infoNode)
+void
+parse_data(InfosetBase *infoset, PState *pstate)
 {
-    const Error *error = handler->visitStartDocument(handler);
+    infoset->erd->parseSelf(infoset, pstate);
+    no_leftover_data(pstate);
+}
 
-    if (!error)
-    {
-        error = walkInfosetNode(handler, infoNode);
-    }
-    if (!error)
-    {
-        error = handler->visitEndDocument(handler);
-    }
+// unparse_infoset - unparse an infoset to an output stream, flush the
+// fractional byte if not done yet, and return any errors in ustate
 
-    return error;
+void
+unparse_infoset(InfosetBase *infoset, UState *ustate)
+{
+    infoset->erd->unparseSelf(infoset, ustate);
+    flush_fractional_byte(ustate);

Review Comment:
   The term we use for this in runtime1 is "fragment byte". I'm not sure how important it is that we are consistent in the different runtimes, so up to you if you want to change it or not.



##########
daffodil-cli/src/main/scala/org/apache/daffodil/cli/Main.scala:
##########
@@ -1618,7 +1618,7 @@ class Main(
                         fail += 1
                         if (testOpts.info() > 0) {
                           STDOUT.println("  Failure Information:")
-                          STDOUT.println(indent(e.getMessage(), 4))
+                          STDOUT.println(indent(e.toString, 4))

Review Comment:
   This seems fine, but your commit mentions the TDML Runner is throwing a NPE? That's probably a bug, any idea what caused it?



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

To unsubscribe, e-mail: commits-unsubscribe@daffodil.apache.org

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


[GitHub] [daffodil] tuxji commented on a diff in pull request #998: Add left over data check to generated C parsers

Posted by "tuxji (via GitHub)" <gi...@apache.org>.
tuxji commented on code in PR #998:
URL: https://github.com/apache/daffodil/pull/998#discussion_r1160970162


##########
daffodil-cli/src/main/scala/org/apache/daffodil/cli/Main.scala:
##########
@@ -1618,7 +1618,7 @@ class Main(
                         fail += 1
                         if (testOpts.info() > 0) {
                           STDOUT.println("  Failure Information:")
-                          STDOUT.println(indent(e.getMessage(), 4))
+                          STDOUT.println(indent(e.toString, 4))

Review Comment:
   Yes, it was a bug in an unreleased code generator's DaffodilTDMLDFDLProcessor.scala which caused TDMLRunner to throw a NPE.  I fixed the bug at its source but wanted to protect Main against a reoccurrence as well.
   



##########
daffodil-codegen-c/src/main/resources/org/apache/daffodil/codegen/c/files/libruntime/unparsers.c:
##########
@@ -465,3 +465,30 @@ unparse_check_bounds(const char *name, size_t count, size_t minOccurs, size_t ma
         ustate->error = &error;
     }
 }
+
+// Flush the fractional byte if not done yet
+
+void
+flush_fractional_byte(UState *ustate)
+{
+    // Skip the flush if we already have an error
+    if (!ustate->error)
+    {
+        // Do we have any unwritten bits left in the fractional byte?
+        if (ustate->numUnwritBits)
+        {
+            // Fill the fractional byte
+            size_t num_bits_fill = BYTE_WIDTH - ustate->numUnwritBits;
+            ustate->unwritBits <<= num_bits_fill;

Review Comment:
   I've made a partial fix.  Now flush_fragment_byte takes a fill_byte parameter and fills the fragment byte from the fill byte before flushing the fragment byte.  However, the calling function, `unparse_infoset` in infoset.c, passes a hardcoded zero fill byte since it doesn't have access to the `dfdl:fillByte` property at this time.



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

To unsubscribe, e-mail: commits-unsubscribe@daffodil.apache.org

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


[GitHub] [daffodil] tuxji commented on a diff in pull request #998: Add left over data check to generated C parsers

Posted by "tuxji (via GitHub)" <gi...@apache.org>.
tuxji commented on code in PR #998:
URL: https://github.com/apache/daffodil/pull/998#discussion_r1160977292


##########
daffodil-codegen-c/src/main/scala/org/apache/daffodil/codegen/c/generators/CodeGeneratorState.scala:
##########
@@ -354,19 +365,20 @@ class CodeGeneratorState(private val root: ElementBase) {
          |// Initialize, parse, and unparse nodes of the infoset
          |
          |$finalImplementation
-         |// Return a root element for parsing or unparsing the infoset
+         |// Get an infoset (optionally clearing it first) for parsing/walking
          |
          |InfosetBase *
-         |rootElement(void)
+         |get_infoset(bool clear_infoset)
          |{
-         |    static bool initialized;
-         |    static $rootName root;
-         |    if (!initialized)
+         |    static $rootName infoset;
+         |
+         |    if (clear_infoset)
          |    {
-         |        ${rootName}_initERD(&root, (InfosetBase *)&root);
-         |        initialized = true;
+         |        memset(&infoset, 0, sizeof(infoset));

Review Comment:
   I added a comment at this location to warn callers that get_infoset(CLEAR_INFOSET) won't free malloc'ed storage for hexBinary prefixed length elements for now.  Writing and calling such extra code didn't seem worth it to me at this time.



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

To unsubscribe, e-mail: commits-unsubscribe@daffodil.apache.org

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


[GitHub] [daffodil] tuxji merged pull request #998: Add left over data check to generated C parsers

Posted by "tuxji (via GitHub)" <gi...@apache.org>.
tuxji merged PR #998:
URL: https://github.com/apache/daffodil/pull/998


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

To unsubscribe, e-mail: commits-unsubscribe@daffodil.apache.org

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


[GitHub] [daffodil] mbeckerle commented on a diff in pull request #998: Add left over data check to generated C parsers

Posted by "mbeckerle (via GitHub)" <gi...@apache.org>.
mbeckerle commented on code in PR #998:
URL: https://github.com/apache/daffodil/pull/998#discussion_r1156145143


##########
daffodil-codegen-c/src/main/resources/org/apache/daffodil/codegen/c/files/libruntime/unparsers.c:
##########
@@ -465,3 +465,30 @@ unparse_check_bounds(const char *name, size_t count, size_t minOccurs, size_t ma
         ustate->error = &error;
     }
 }
+
+// Flush any fractional bits not written yet
+
+void
+flush_ustate(UState *ustate)

Review Comment:
   Hmmm. It's a fraction of a byte yes? Not sure what a fractional bit is. I get that you mean one of the bits of the fractional byte, but I would call it flush_fractional_byte. 



##########
daffodil-codegen-c/src/main/resources/org/apache/daffodil/codegen/c/files/libruntime/parsers.c:
##########
@@ -594,3 +594,34 @@ parse_check_bounds(const char *name, size_t count, size_t minOccurs, size_t maxO
         pstate->error = &error;
     }
 }
+
+// Check for any data left over after end of parse
+
+void
+check_pstate(PState *pstate)
+{
+    // Skip the check if we already have an error
+    if (!pstate->error)
+    {
+        // Check for any unread bits left in pstate's fractional byte
+        if (pstate->unreadLen)

Review Comment:
   Consider maybe renaming unreadLen to unreadBitsLen to make it clear this is in units of bits.
   
   Since most library and other C code works in bytes I think it's useful to specify Bits when that's the units. 
   



##########
daffodil-codegen-c/src/test/examples/NestedUnion/generated_code.c:
##########
@@ -94,7 +94,7 @@ static const ERD foo_data_NestedUnionType_ERD = {
     },
     COMPLEX, // typeCode
     3, // numChildren
-    foo_data_NestedUnionType__offsets, // offsets
+    foo_data_NestedUnionType__childrenOffsets, // childrenOffsets

Review Comment:
   comments seem redundant with identifier names. Here and also in a few other spots below. 
   
   The comments add value for the things that aren't identifiers that end in exactly a suffix that matches the comment.  But for the ones where there is an identifiers, they really are just clutter. 



##########
daffodil-codegen-c/src/main/scala/org/apache/daffodil/codegen/c/generators/CodeGeneratorState.scala:
##########
@@ -400,7 +427,7 @@ class CodeGeneratorState(private val root: ElementBase) {
       if (numChildren > 0)
         s"""static const $C ${C}_compute_offsets;
          |
-         |static const size_t ${C}_offsets[$count] = {
+         |static const size_t ${C}_childrenOffsets[$count] = {

Review Comment:
   You didn't snake-case numerous identifiers like childrenOffsets, so I assume that change was not intended to be pervasive for all identifiers. 



##########
daffodil-codegen-c/src/main/resources/org/apache/daffodil/codegen/c/files/libruntime/parsers.c:
##########
@@ -594,3 +594,34 @@ parse_check_bounds(const char *name, size_t count, size_t minOccurs, size_t maxO
         pstate->error = &error;
     }
 }
+
+// Check for any data left over after end of parse
+
+void
+check_pstate(PState *pstate)

Review Comment:
   Is this going to generalize to other checks of the pstate? If it's is just about the left-over-data, then rename accordingly. 
   
   Also consider since this is about after the end of the parse, why  not "check_final_pstate" to reflect this is not an internal consistency check, but for checking at the end. 



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

To unsubscribe, e-mail: commits-unsubscribe@daffodil.apache.org

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


[GitHub] [daffodil] tuxji commented on a diff in pull request #998: Add left over data check to generated C parsers

Posted by "tuxji (via GitHub)" <gi...@apache.org>.
tuxji commented on code in PR #998:
URL: https://github.com/apache/daffodil/pull/998#discussion_r1156254199


##########
daffodil-codegen-c/src/main/resources/org/apache/daffodil/codegen/c/files/libruntime/parsers.c:
##########
@@ -594,3 +594,34 @@ parse_check_bounds(const char *name, size_t count, size_t minOccurs, size_t maxO
         pstate->error = &error;
     }
 }
+
+// Check for any data left over after end of parse
+
+void
+check_pstate(PState *pstate)

Review Comment:
   No, this function probably won't do anything else, so I've renamed it accordingly to `no_leftover_data`.



##########
daffodil-codegen-c/src/main/resources/org/apache/daffodil/codegen/c/files/libruntime/parsers.c:
##########
@@ -594,3 +594,34 @@ parse_check_bounds(const char *name, size_t count, size_t minOccurs, size_t maxO
         pstate->error = &error;
     }
 }
+
+// Check for any data left over after end of parse
+
+void
+check_pstate(PState *pstate)
+{
+    // Skip the check if we already have an error
+    if (!pstate->error)
+    {
+        // Check for any unread bits left in pstate's fractional byte
+        if (pstate->unreadLen)

Review Comment:
   I've renamed `unreadLen` to `numUnreadBits`/`numUnwritBits`, which better matches a local variable called `num_bits` in the C code using them.



##########
daffodil-codegen-c/src/main/scala/org/apache/daffodil/codegen/c/generators/CodeGeneratorState.scala:
##########
@@ -400,7 +427,7 @@ class CodeGeneratorState(private val root: ElementBase) {
       if (numChildren > 0)
         s"""static const $C ${C}_compute_offsets;
          |
-         |static const size_t ${C}_offsets[$count] = {
+         |static const size_t ${C}_childrenOffsets[$count] = {

Review Comment:
   Yes, I didn't want to do a comprehensive job of renaming in this PR.  The remaining camel case identifiers are in "types", which include struct field names as well, and it sort of makes sense to keep them closer to Daffodil's camel case style since both the Daffodil and C types talk about similar parsing/unparsing/metadata data.



##########
daffodil-codegen-c/src/test/examples/NestedUnion/generated_code.c:
##########
@@ -94,7 +94,7 @@ static const ERD foo_data_NestedUnionType_ERD = {
     },
     COMPLEX, // typeCode
     3, // numChildren
-    foo_data_NestedUnionType__offsets, // offsets
+    foo_data_NestedUnionType__childrenOffsets, // childrenOffsets

Review Comment:
   Good point.  I've removed redundant comments where they suffix similar identifiers and left comments where they suffix NULLs.



##########
daffodil-codegen-c/src/main/resources/org/apache/daffodil/codegen/c/files/libruntime/unparsers.c:
##########
@@ -465,3 +465,30 @@ unparse_check_bounds(const char *name, size_t count, size_t minOccurs, size_t ma
         ustate->error = &error;
     }
 }
+
+// Flush any fractional bits not written yet
+
+void
+flush_ustate(UState *ustate)

Review Comment:
   Yes, I was collapsing "fractional byte's bits" to two words.  I've changed "fractional bits" back to "fractional byte" in comments and renamed `flush_ustate` to `flush_fractional_byte`.



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

To unsubscribe, e-mail: commits-unsubscribe@daffodil.apache.org

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


[GitHub] [daffodil] tuxji commented on a diff in pull request #998: Add left over data check to generated C parsers

Posted by "tuxji (via GitHub)" <gi...@apache.org>.
tuxji commented on code in PR #998:
URL: https://github.com/apache/daffodil/pull/998#discussion_r1160974836


##########
daffodil-codegen-c/src/main/resources/org/apache/daffodil/codegen/c/files/libruntime/infoset.c:
##########
@@ -123,64 +124,55 @@ get_erd_ns(const ERD *erd)
     return erd->namedQName.ns;
 }
 
-// walkInfoset - walk each node of an infoset and call
-// VisitEventHandler methods
+// parse_data - parse an input stream into an infoset, check for
+// leftover data, and return any errors in pstate
 
-const Error *
-walkInfoset(const VisitEventHandler *handler, const InfosetBase *infoNode)
+void
+parse_data(InfosetBase *infoset, PState *pstate)
 {
-    const Error *error = handler->visitStartDocument(handler);
+    infoset->erd->parseSelf(infoset, pstate);
+    no_leftover_data(pstate);
+}
 
-    if (!error)
-    {
-        error = walkInfosetNode(handler, infoNode);
-    }
-    if (!error)
-    {
-        error = handler->visitEndDocument(handler);
-    }
+// unparse_infoset - unparse an infoset to an output stream, flush the
+// fractional byte if not done yet, and return any errors in ustate
 
-    return error;
+void
+unparse_infoset(InfosetBase *infoset, UState *ustate)
+{
+    infoset->erd->unparseSelf(infoset, ustate);
+    flush_fractional_byte(ustate);

Review Comment:
   I'm glad you told me.  I think fragment byte is a much better term than fractional byte and I've made the change.



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

To unsubscribe, e-mail: commits-unsubscribe@daffodil.apache.org

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


[GitHub] [daffodil] tuxji commented on a diff in pull request #998: Add left over data check to generated C parsers

Posted by "tuxji (via GitHub)" <gi...@apache.org>.
tuxji commented on code in PR #998:
URL: https://github.com/apache/daffodil/pull/998#discussion_r1160975109


##########
daffodil-codegen-c/src/main/scala/org/apache/daffodil/codegen/c/DaffodilCExamplesGenerator.scala:
##########
@@ -81,19 +81,23 @@ object DaffodilCExamplesGenerator {
     val nestedRootName = Some("NestedUnion")
     val padTestSchema = schemaDir / "padtest.dfdl.xsd"
     val padTestRootName = None
+    val simpleSchema = schemaDir / "simple.dfdl.xsd"
+    val simpleRootName = Some("simple-byte")
     val variableLenSchema = schemaDir / "variablelen.dfdl.xsd"
     val variableLenRootName = Some("expressionElement")
 
     val examplesDir = os.Path(args(0))
     val exNumsExampleDir = examplesDir / "ex_nums"
     val nestedExampleDir = examplesDir / "NestedUnion"
     val padTestExampleDir = examplesDir / "padtest"
+    val simpleExampleDir = examplesDir / "simple"
     val variableLenExampleDir = examplesDir / "variablelen"

Review Comment:
   Good idea.  I've replaced the multiple schema, root, dir variables with the Array of tuples.



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

To unsubscribe, e-mail: commits-unsubscribe@daffodil.apache.org

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


[GitHub] [daffodil] tuxji commented on a diff in pull request #998: Add left over data check to generated C parsers

Posted by "tuxji (via GitHub)" <gi...@apache.org>.
tuxji commented on code in PR #998:
URL: https://github.com/apache/daffodil/pull/998#discussion_r1160977679


##########
daffodil-codegen-c/src/main/scala/org/apache/daffodil/codegen/c/generators/CodeGeneratorState.scala:
##########
@@ -727,22 +778,32 @@ class CodeGeneratorState(private val root: ElementBase) {
 
   // Recursively builds a hopefully unique name using the given StringBuilder
   private def buildName(sc: SchemaComponent, sb: StringBuilder): StringBuilder = {
+    // Append schema component's name
     sc match {
       case eb: ElementBase => sb ++= eb.namedQName.local += '_'
       case gd: GlobalElementDecl => sb ++= gd.namedQName.local += '_'
       case ct: GlobalComplexTypeDef => sb ++= ct.namedQName.local += '_'
       case _ => // don't include other schema components in qualified name
     }
+    // Recursively append parent schema components' names
     sc.optLexicalParent.foreach {
       buildName(_, sb)
     }
+    // Ensure name contains only legal characters for C identifiers
+    lazy val legalChars: immutable.Set[Char] =
+      Set('_') ++ ('a' to 'z') ++ ('A' to 'Z') ++ ('0' to '9')
+    for (i <- sb.indices) {
+      if (!legalChars.contains(sb.charAt(i))) {
+        sb.setCharAt(i, '_')
+      }
+    }

Review Comment:
   Yes, I've refactored the duplicate code for legal C identifiers into a single `makeLegalForC` function.



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

To unsubscribe, e-mail: commits-unsubscribe@daffodil.apache.org

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