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 2022/11/18 19:51:38 UTC

[GitHub] [daffodil] jadams-tresys opened a new pull request, #878: Implement a encode/decodeEXI command to the CLI

jadams-tresys opened a new pull request, #878:
URL: https://github.com/apache/daffodil/pull/878

   This command simply encodes an existing XML file to EXI using Exificient. It will do a schema aware encoding if a schema is provided. On the other side the decodeEXI command will decode an existing EXI file to a plain text XML file.
   
   This is a useful feature as there isn't an easy existing way to perform these conversions using a schema aware encoding as it requires the use of the DaffodilXMLEntityResolver for anything more than a basic schema.
   
   DAFFODIL-2750


-- 
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 #878: Implement a encode/decodeEXI command to the CLI

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on code in PR #878:
URL: https://github.com/apache/daffodil/pull/878#discussion_r1026843932


##########
daffodil-cli/src/main/scala/org/apache/daffodil/Main.scala:
##########
@@ -514,12 +516,46 @@ class CLIConf(arguments: Array[String]) extends scallop.ScallopConf(arguments) {
     requireSubcommand()
   }
 
+  // Encode EXI Subcommand Options
+  object encodeEXI extends scallop.Subcommand("encodeEXI") {
+    banner("""|Usage: daffodil encodeEXI [-s <schema>] [-o <output>] [infile]

Review Comment:
   Thoughts on having a single subcommand `exi` that can both encode and decode? For example, the `base64` command encodes by default and decodes with the `-d` option. Maybe something similar?
   
   I almost wonder if this wants to be a separate utility, e.g. `dfdlexi`? Might not be worth the effort.
   



##########
daffodil-cli/src/it/scala/org/apache/daffodil/exi/TestEXIEncodeDecode.scala:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.saving
+
+import org.junit.Test
+import scala.xml.{Node, XML}
+
+import org.apache.daffodil.CLI.Util._
+import org.apache.daffodil.Main.ExitCode
+import org.apache.daffodil.xml.XMLUtils
+
+class TestCLIEncodeDecodeEXI {
+
+  @Test def test_3017_CLI_Encode_Decode_EXI_SA(): Unit = {
+    val schema = path("daffodil-test/src/test/resources/org/apache/daffodil/section06/namespaces/multi_base_15.dfdl.xsd")
+
+    withTempFile { tempEXI =>
+
+      // Create initial XML infoset
+      var infosetXML: Node = null
+      var encodedMD5: String = null
+      runCLI(args"parse -s $schema") { cli =>
+        cli.send("test", inputDone = true)
+        val res = cli.expectEOF
+        infosetXML = XML.loadString(res.getBefore)
+      } (ExitCode.Success)

Review Comment:
   Do we even need to parse? Can we just use one of the infosets in daffod-cli? These tests are much faster than they used to be but there's still some overhead in these integration tests. The less we can do the better.



##########
daffodil-cli/src/it/scala/org/apache/daffodil/exi/TestEXIEncodeDecode.scala:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.saving
+
+import org.junit.Test
+import scala.xml.{Node, XML}
+
+import org.apache.daffodil.CLI.Util._
+import org.apache.daffodil.Main.ExitCode
+import org.apache.daffodil.xml.XMLUtils
+
+class TestCLIEncodeDecodeEXI {
+
+  @Test def test_3017_CLI_Encode_Decode_EXI_SA(): Unit = {
+    val schema = path("daffodil-test/src/test/resources/org/apache/daffodil/section06/namespaces/multi_base_15.dfdl.xsd")
+
+    withTempFile { tempEXI =>
+
+      // Create initial XML infoset
+      var infosetXML: Node = null
+      var encodedMD5: String = null
+      runCLI(args"parse -s $schema") { cli =>
+        cli.send("test", inputDone = true)
+        val res = cli.expectEOF
+        infosetXML = XML.loadString(res.getBefore)
+      } (ExitCode.Success)
+
+      // Encode infoset to schema aware EXI
+      runCLI(args"encodeEXI -s $schema -o $tempEXI") { cli =>
+        cli.sendLine(infosetXML.toString, inputDone = true)
+        encodedMD5 = md5sum(tempEXI)

Review Comment:
   This should happen outside of runCLI. The encodeEXI command isn't guaranteed to be done by the time md5sum is called.



##########
daffodil-cli/src/it/scala/org/apache/daffodil/exi/TestEXIEncodeDecode.scala:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.saving
+
+import org.junit.Test
+import scala.xml.{Node, XML}
+
+import org.apache.daffodil.CLI.Util._
+import org.apache.daffodil.Main.ExitCode
+import org.apache.daffodil.xml.XMLUtils
+
+class TestCLIEncodeDecodeEXI {
+
+  @Test def test_3017_CLI_Encode_Decode_EXI_SA(): Unit = {
+    val schema = path("daffodil-test/src/test/resources/org/apache/daffodil/section06/namespaces/multi_base_15.dfdl.xsd")
+
+    withTempFile { tempEXI =>
+
+      // Create initial XML infoset
+      var infosetXML: Node = null
+      var encodedMD5: String = null
+      runCLI(args"parse -s $schema") { cli =>
+        cli.send("test", inputDone = true)
+        val res = cli.expectEOF
+        infosetXML = XML.loadString(res.getBefore)
+      } (ExitCode.Success)
+
+      // Encode infoset to schema aware EXI
+      runCLI(args"encodeEXI -s $schema -o $tempEXI") { cli =>
+        cli.sendLine(infosetXML.toString, inputDone = true)
+        encodedMD5 = md5sum(tempEXI)
+      } (ExitCode.Success)
+
+      // Decode EXI to XML and compare against original XML infoset
+      runCLI(args"decodeEXI -s $schema $tempEXI") { cli =>
+        val res = cli.expectEOF
+        val resultNode = XML.loadString(res.getBefore)

Review Comment:
   I'm not sure how reliable getBefore is. I'm prety sure it's not guaranteed to return the full output. Instead, I'd recommend creating another tempFile that you decode to and them read that file and compare.
   
   I htink this expectEOF + getBefore is a bit of a hack and probably should be avoided. I've removed most uses of it.



##########
daffodil-cli/src/main/scala/org/apache/daffodil/Main.scala:
##########
@@ -1482,6 +1518,74 @@ object Main {
         }
       }
 
+      case Some(conf.encodeEXI) => {
+        val encodeOpts = conf.encodeEXI
+        val channel = encodeOpts.output.toOption match {
+          case Some("-") | None => Channels.newChannel(STDOUT)
+          case Some(file) => new FileOutputStream(file).getChannel()
+        }
+        val output = Channels.newOutputStream(channel)
+
+        val inputStream = encodeOpts.infile.toOption match {
+          case Some("-") | None => STDIN
+          case Some(file) => {
+            val f = new File(file)
+            new FileInputStream(f)
+          }
+        }
+        val input = new InputSource(inputStream)
+
+        val exiFactory = {
+          if (encodeOpts.schema.isDefined)
+            getExiFactoryOpt(InfosetType.EXISA, encodeOpts.schema.toOption)
+          else
+            getExiFactoryOpt(InfosetType.EXI, encodeOpts.schema.toOption)
+        }
+        val exiResult = new EXIResult(exiFactory.get)
+        exiResult.setOutputStream(output)
+
+        val reader = XMLReaderFactory.createXMLReader()
+        reader.setContentHandler(exiResult.getHandler)
+        reader.parse(input)
+        output.close

Review Comment:
   Do we need to catch any exceptions and return a exit code? Otherwise any exceptions will probably be reported as a bug, when really it should be reported as "invalid EXI/XML" or something.



##########
daffodil-cli/src/main/scala/org/apache/daffodil/Main.scala:
##########
@@ -1482,6 +1518,74 @@ object Main {
         }
       }
 
+      case Some(conf.encodeEXI) => {
+        val encodeOpts = conf.encodeEXI
+        val channel = encodeOpts.output.toOption match {
+          case Some("-") | None => Channels.newChannel(STDOUT)
+          case Some(file) => new FileOutputStream(file).getChannel()
+        }
+        val output = Channels.newOutputStream(channel)
+
+        val inputStream = encodeOpts.infile.toOption match {
+          case Some("-") | None => STDIN
+          case Some(file) => {
+            val f = new File(file)
+            new FileInputStream(f)
+          }
+        }
+        val input = new InputSource(inputStream)
+
+        val exiFactory = {
+          if (encodeOpts.schema.isDefined)
+            getExiFactoryOpt(InfosetType.EXISA, encodeOpts.schema.toOption)
+          else
+            getExiFactoryOpt(InfosetType.EXI, encodeOpts.schema.toOption)
+        }

Review Comment:
   Note that this has all been refactored in PR #873. We'll have to fix some merge conflicts depending on which is merged first. I think it should be fairly easy to update this to use the new stuff either way since it's just a matter of making the EXIFactory publicly available.



##########
daffodil-cli/src/it/scala/org/apache/daffodil/exi/TestEXIEncodeDecode.scala:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.saving
+
+import org.junit.Test
+import scala.xml.{Node, XML}
+
+import org.apache.daffodil.CLI.Util._
+import org.apache.daffodil.Main.ExitCode
+import org.apache.daffodil.xml.XMLUtils
+
+class TestCLIEncodeDecodeEXI {
+
+  @Test def test_3017_CLI_Encode_Decode_EXI_SA(): Unit = {
+    val schema = path("daffodil-test/src/test/resources/org/apache/daffodil/section06/namespaces/multi_base_15.dfdl.xsd")
+
+    withTempFile { tempEXI =>
+
+      // Create initial XML infoset
+      var infosetXML: Node = null
+      var encodedMD5: String = null
+      runCLI(args"parse -s $schema") { cli =>
+        cli.send("test", inputDone = true)
+        val res = cli.expectEOF
+        infosetXML = XML.loadString(res.getBefore)
+      } (ExitCode.Success)
+
+      // Encode infoset to schema aware EXI
+      runCLI(args"encodeEXI -s $schema -o $tempEXI") { cli =>
+        cli.sendLine(infosetXML.toString, inputDone = true)
+        encodedMD5 = md5sum(tempEXI)
+      } (ExitCode.Success)
+
+      // Decode EXI to XML and compare against original XML infoset
+      runCLI(args"decodeEXI -s $schema $tempEXI") { cli =>
+        val res = cli.expectEOF
+        val resultNode = XML.loadString(res.getBefore)
+        XMLUtils.compareAndReport(infosetXML, resultNode)

Review Comment:
   Like before, this should happen outside of runCLI when we are guaranteed the CLI is done with its work.



##########
daffodil-cli/src/it/scala/org/apache/daffodil/exi/TestEXIEncodeDecode.scala:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.saving
+
+import org.junit.Test
+import scala.xml.{Node, XML}
+
+import org.apache.daffodil.CLI.Util._
+import org.apache.daffodil.Main.ExitCode
+import org.apache.daffodil.xml.XMLUtils
+
+class TestCLIEncodeDecodeEXI {
+
+  @Test def test_3017_CLI_Encode_Decode_EXI_SA(): Unit = {
+    val schema = path("daffodil-test/src/test/resources/org/apache/daffodil/section06/namespaces/multi_base_15.dfdl.xsd")
+
+    withTempFile { tempEXI =>
+
+      // Create initial XML infoset
+      var infosetXML: Node = null
+      var encodedMD5: String = null
+      runCLI(args"parse -s $schema") { cli =>
+        cli.send("test", inputDone = true)
+        val res = cli.expectEOF
+        infosetXML = XML.loadString(res.getBefore)
+      } (ExitCode.Success)
+
+      // Encode infoset to schema aware EXI
+      runCLI(args"encodeEXI -s $schema -o $tempEXI") { cli =>
+        cli.sendLine(infosetXML.toString, inputDone = true)
+        encodedMD5 = md5sum(tempEXI)
+      } (ExitCode.Success)
+
+      // Decode EXI to XML and compare against original XML infoset
+      runCLI(args"decodeEXI -s $schema $tempEXI") { cli =>
+        val res = cli.expectEOF
+        val resultNode = XML.loadString(res.getBefore)
+        XMLUtils.compareAndReport(infosetXML, resultNode)
+      } (ExitCode.Success)
+
+      // Generate the infoset in schema aware EXI directly
+      runCLI(args"parse -s $schema -I exisa -o $tempEXI") { cli =>
+        cli.send("test", inputDone = true)
+      } (ExitCode.Success)

Review Comment:
   Like before, can we reuse an existing EXI file in the repo and avoid he overhead with more parsing?



##########
daffodil-cli/src/it/scala/org/apache/daffodil/exi/TestEXIEncodeDecode.scala:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.saving
+
+import org.junit.Test
+import scala.xml.{Node, XML}
+
+import org.apache.daffodil.CLI.Util._
+import org.apache.daffodil.Main.ExitCode
+import org.apache.daffodil.xml.XMLUtils
+
+class TestCLIEncodeDecodeEXI {
+
+  @Test def test_3017_CLI_Encode_Decode_EXI_SA(): Unit = {
+    val schema = path("daffodil-test/src/test/resources/org/apache/daffodil/section06/namespaces/multi_base_15.dfdl.xsd")
+
+    withTempFile { tempEXI =>
+
+      // Create initial XML infoset
+      var infosetXML: Node = null
+      var encodedMD5: String = null
+      runCLI(args"parse -s $schema") { cli =>
+        cli.send("test", inputDone = true)
+        val res = cli.expectEOF
+        infosetXML = XML.loadString(res.getBefore)
+      } (ExitCode.Success)
+
+      // Encode infoset to schema aware EXI
+      runCLI(args"encodeEXI -s $schema -o $tempEXI") { cli =>
+        cli.sendLine(infosetXML.toString, inputDone = true)
+        encodedMD5 = md5sum(tempEXI)
+      } (ExitCode.Success)
+
+      // Decode EXI to XML and compare against original XML infoset
+      runCLI(args"decodeEXI -s $schema $tempEXI") { cli =>
+        val res = cli.expectEOF
+        val resultNode = XML.loadString(res.getBefore)
+        XMLUtils.compareAndReport(infosetXML, resultNode)
+      } (ExitCode.Success)
+
+      // Generate the infoset in schema aware EXI directly
+      runCLI(args"parse -s $schema -I exisa -o $tempEXI") { cli =>
+        cli.send("test", inputDone = true)
+      } (ExitCode.Success)
+
+      // Decode the parsed EXI infoset to XML and compare against original XML
+      // infoset
+      runCLI(args"decodeEXI -s $schema $tempEXI") { cli =>
+        val res = cli.expectEOF
+        val resultNode = XML.loadString(res.getBefore)
+        XMLUtils.compareAndReport(infosetXML, resultNode)
+      } (ExitCode.Success)
+    }
+  }
+
+  @Test def test_3017_CLI_Encode_Decode_EXI(): Unit = {
+    val inputXML = <person><name>Edward</name><age>42</age></person>
+
+    withTempFile { tempEXI =>
+      runCLI(args"encodeEXI -o $tempEXI") { cli =>
+        cli.sendLine(inputXML.toString, inputDone = true)
+      } (ExitCode.Success)
+
+      runCLI(args"decodeEXI $tempEXI") { cli =>
+        val res = cli.expectEOF
+        val resultNode = XML.loadString(res.getBefore)
+        XMLUtils.compareAndReport(inputXML, resultNode)

Review Comment:
   Move these lines outside of runCLI.



##########
daffodil-cli/src/it/scala/org/apache/daffodil/exi/TestEXIEncodeDecode.scala:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.saving
+
+import org.junit.Test
+import scala.xml.{Node, XML}
+
+import org.apache.daffodil.CLI.Util._
+import org.apache.daffodil.Main.ExitCode
+import org.apache.daffodil.xml.XMLUtils
+
+class TestCLIEncodeDecodeEXI {
+
+  @Test def test_3017_CLI_Encode_Decode_EXI_SA(): Unit = {
+    val schema = path("daffodil-test/src/test/resources/org/apache/daffodil/section06/namespaces/multi_base_15.dfdl.xsd")
+
+    withTempFile { tempEXI =>
+
+      // Create initial XML infoset
+      var infosetXML: Node = null
+      var encodedMD5: String = null
+      runCLI(args"parse -s $schema") { cli =>
+        cli.send("test", inputDone = true)
+        val res = cli.expectEOF
+        infosetXML = XML.loadString(res.getBefore)
+      } (ExitCode.Success)
+
+      // Encode infoset to schema aware EXI
+      runCLI(args"encodeEXI -s $schema -o $tempEXI") { cli =>
+        cli.sendLine(infosetXML.toString, inputDone = true)
+        encodedMD5 = md5sum(tempEXI)
+      } (ExitCode.Success)
+
+      // Decode EXI to XML and compare against original XML infoset
+      runCLI(args"decodeEXI -s $schema $tempEXI") { cli =>
+        val res = cli.expectEOF
+        val resultNode = XML.loadString(res.getBefore)
+        XMLUtils.compareAndReport(infosetXML, resultNode)
+      } (ExitCode.Success)
+
+      // Generate the infoset in schema aware EXI directly
+      runCLI(args"parse -s $schema -I exisa -o $tempEXI") { cli =>
+        cli.send("test", inputDone = true)
+      } (ExitCode.Success)
+
+      // Decode the parsed EXI infoset to XML and compare against original XML
+      // infoset
+      runCLI(args"decodeEXI -s $schema $tempEXI") { cli =>
+        val res = cli.expectEOF
+        val resultNode = XML.loadString(res.getBefore)

Review Comment:
   Same before. I suggest using withTempFile and outputting to that and then compare the file instead of trying to capture all of stdout.



##########
daffodil-cli/src/main/scala/org/apache/daffodil/Main.scala:
##########
@@ -1482,6 +1518,74 @@ object Main {
         }
       }
 
+      case Some(conf.encodeEXI) => {
+        val encodeOpts = conf.encodeEXI
+        val channel = encodeOpts.output.toOption match {
+          case Some("-") | None => Channels.newChannel(STDOUT)
+          case Some(file) => new FileOutputStream(file).getChannel()
+        }
+        val output = Channels.newOutputStream(channel)
+
+        val inputStream = encodeOpts.infile.toOption match {
+          case Some("-") | None => STDIN
+          case Some(file) => {
+            val f = new File(file)
+            new FileInputStream(f)
+          }
+        }
+        val input = new InputSource(inputStream)
+
+        val exiFactory = {
+          if (encodeOpts.schema.isDefined)
+            getExiFactoryOpt(InfosetType.EXISA, encodeOpts.schema.toOption)
+          else
+            getExiFactoryOpt(InfosetType.EXI, encodeOpts.schema.toOption)
+        }
+        val exiResult = new EXIResult(exiFactory.get)
+        exiResult.setOutputStream(output)
+
+        val reader = XMLReaderFactory.createXMLReader()
+        reader.setContentHandler(exiResult.getHandler)
+        reader.parse(input)
+        output.close
+
+        ExitCode.Success
+      }
+
+      case Some(conf.decodeEXI) => {
+        val decodeOpts = conf.decodeEXI
+        val channel = decodeOpts.output.toOption match {
+          case Some("-") | None => Channels.newChannel(STDOUT)
+          case Some(file) => new FileOutputStream(file).getChannel()
+        }
+        val output = Channels.newOutputStream(channel)
+
+        val inputStream = decodeOpts.infile.toOption match {
+          case Some("-") | None => STDIN
+          case Some(file) => {
+            val f = new File(file)
+            new FileInputStream(f)
+          }
+        }
+        val input = new InputSource(inputStream)
+
+        val exiFactory = {
+          if (decodeOpts.schema.isDefined)
+            getExiFactoryOpt(InfosetType.EXISA, decodeOpts.schema.toOption)
+          else
+            getExiFactoryOpt(InfosetType.EXI, decodeOpts.schema.toOption)
+        }
+        val exiSource = new EXISource(exiFactory.get)
+        exiSource.setInputSource(input)
+
+        val result = new StreamResult(output)
+        val tf = TransformerFactory.newInstance()
+        val transformer = tf.newTransformer
+        transformer.transform(exiSource, result)
+
+        ExitCode.Success

Review Comment:
   Same comments above about potential merge conflicts and exit code.



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

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 #878: Implement a encode/decodeEXI command to the CLI

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on code in PR #878:
URL: https://github.com/apache/daffodil/pull/878#discussion_r1033514969


##########
daffodil-cli/src/main/scala/org/apache/daffodil/Main.scala:
##########
@@ -1482,6 +1518,74 @@ object Main {
         }
       }
 
+      case Some(conf.encodeEXI) => {
+        val encodeOpts = conf.encodeEXI
+        val channel = encodeOpts.output.toOption match {
+          case Some("-") | None => Channels.newChannel(STDOUT)
+          case Some(file) => new FileOutputStream(file).getChannel()
+        }
+        val output = Channels.newOutputStream(channel)
+
+        val inputStream = encodeOpts.infile.toOption match {
+          case Some("-") | None => STDIN
+          case Some(file) => {
+            val f = new File(file)
+            new FileInputStream(f)
+          }
+        }
+        val input = new InputSource(inputStream)
+
+        val exiFactory = {
+          if (encodeOpts.schema.isDefined)
+            getExiFactoryOpt(InfosetType.EXISA, encodeOpts.schema.toOption)
+          else
+            getExiFactoryOpt(InfosetType.EXI, encodeOpts.schema.toOption)
+        }

Review Comment:
   I've merged that PR.
   
   I think the changes you'll need are to make the `exiFactory` member of the `EXIInfosetHandler` public here:
   
   https://github.com/apache/daffodil/blob/main/daffodil-cli/src/main/scala/org/apache/daffodil/InfosetTypes.scala#L676
   
   Then you can call `getInfosetHandler` and access that `exiFactory` member. This only changes how you get the factory though, so I think everything else should be the same.



-- 
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] jadams-tresys commented on a diff in pull request #878: Implement a encode/decodeEXI command to the CLI

Posted by GitBox <gi...@apache.org>.
jadams-tresys commented on code in PR #878:
URL: https://github.com/apache/daffodil/pull/878#discussion_r1034746479


##########
daffodil-cli/src/it/scala/org/apache/daffodil/exi/TestEXIEncodeDecode.scala:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.saving
+
+import org.junit.Test
+import scala.xml.XML
+import java.nio.file.Paths
+
+import org.apache.daffodil.CLI.Util._
+import org.apache.daffodil.Main.ExitCode
+import org.apache.daffodil.xml.XMLUtils
+
+class TestCLIEncodeDecodeEXI {
+
+  @Test def test_3017_CLI_Encode_Decode_EXI_SA(): Unit = {
+    val schema = path("daffodil-test/src/test/resources/org/apache/daffodil/usertests/Book2.dfdl.xsd")
+    val infosetPath = path("daffodil-test/src/test/resources/org/apache/daffodil/usertests/test_Book2.expected.xml")
+    val infosetXML = XML.loadFile(infosetPath.toFile)
+
+    withTempDir { tempDir =>
+      val tempEXI = Paths.get(tempDir.toString, "temp.exi")
+      val tempXML = Paths.get(tempDir.toString, "temp.xml")
+
+      // Encode infoset to schema aware EXI
+      runCLI(args"exi -s $schema -o $tempEXI") { cli =>
+        cli.sendLine(infosetXML.toString, inputDone = true)
+      } (ExitCode.Success)
+
+      // Decode EXI to XML and compare against original XML infoset
+      runCLI(args"exi -d -s $schema -o $tempXML $tempEXI") { cli =>
+      } (ExitCode.Success)
+
+      val resultNode = XML.loadFile(tempXML.toFile)
+      XMLUtils.compareAndReport(infosetXML, resultNode)
+    }
+  }
+
+  @Test def test_3017_CLI_Encode_Decode_EXI(): Unit = {
+    val inputXML = <person><name>Edward</name><age>42</age></person>
+
+    withTempDir { tempDir =>
+      val tempEXI = Paths.get(tempDir.toString, "temp.exi")
+      val tempXML = Paths.get(tempDir.toString, "temp.xml")
+
+      runCLI(args"exi -o $tempEXI") { cli =>
+        cli.sendLine(inputXML.toString, inputDone = true)
+      } (ExitCode.Success)
+
+      runCLI(args"exi -d -o $tempXML $tempEXI") { cli =>
+      } (ExitCode.Success)
+
+      val resultNode = XML.loadFile(tempXML.toFile)
+      XMLUtils.compareAndReport(inputXML, resultNode)
+    }
+  }
+
+  @Test def test_3017_CLI_EncodeBadFile_EXI(): Unit = {
+    val badXML = path("daffodil-test/src/test/resources/org/apache/daffodil/usertests/Book2.csv")
+
+    runCLI(args"exi $badXML") { cli =>
+      cli.expectErr("Error parsing input XML")
+    } (ExitCode.Failure)
+  }
+
+  @Test def test_3017_CLI_DecodeBadFile_EXI(): Unit = {
+    val badEXI = path("daffodil-test/src/test/resources/org/apache/daffodil/usertests/Book2.csv")
+
+    runCLI(args"exi -d $badEXI") { cli =>
+      cli.expectErr("No valid EXI document")
+    } (ExitCode.Failure)
+  }
+
+  @Test def test_3017_CLI_LoadBadSchema_EXI(): Unit = {
+    val badSchema = path("daffodil-test/src/test/resources/org/apache/daffodil/usertests/Book2.csv")
+
+    runCLI(args"exi -s $badSchema") { cli =>
+      cli.expectErr("Error creating EXI grammar for the supplied schema")
+    } (ExitCode.Failure)
+  }
+}

Review Comment:
   Number 1 will gladly encode a file with a mis-matched shcema and can even save space assuming it finds common elements I think.  It also decodes as expected.
   
   A test for number 2 has been added.



-- 
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] jadams-tresys merged pull request #878: Implement a encode/decodeEXI command to the CLI

Posted by GitBox <gi...@apache.org>.
jadams-tresys merged PR #878:
URL: https://github.com/apache/daffodil/pull/878


-- 
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] jadams-tresys commented on a diff in pull request #878: Implement a encode/decodeEXI command to the CLI

Posted by GitBox <gi...@apache.org>.
jadams-tresys commented on code in PR #878:
URL: https://github.com/apache/daffodil/pull/878#discussion_r1030425122


##########
daffodil-cli/src/main/scala/org/apache/daffodil/Main.scala:
##########
@@ -1482,6 +1518,74 @@ object Main {
         }
       }
 
+      case Some(conf.encodeEXI) => {
+        val encodeOpts = conf.encodeEXI
+        val channel = encodeOpts.output.toOption match {
+          case Some("-") | None => Channels.newChannel(STDOUT)
+          case Some(file) => new FileOutputStream(file).getChannel()
+        }
+        val output = Channels.newOutputStream(channel)
+
+        val inputStream = encodeOpts.infile.toOption match {
+          case Some("-") | None => STDIN
+          case Some(file) => {
+            val f = new File(file)
+            new FileInputStream(f)
+          }
+        }
+        val input = new InputSource(inputStream)
+
+        val exiFactory = {
+          if (encodeOpts.schema.isDefined)
+            getExiFactoryOpt(InfosetType.EXISA, encodeOpts.schema.toOption)
+          else
+            getExiFactoryOpt(InfosetType.EXI, encodeOpts.schema.toOption)
+        }

Review Comment:
   I'll let you get your commit merged before pushing mine up.  I'll take care of resolving any conflicts that come up.



-- 
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 #878: Implement a encode/decodeEXI command to the CLI

Posted by GitBox <gi...@apache.org>.
tuxji commented on code in PR #878:
URL: https://github.com/apache/daffodil/pull/878#discussion_r1033946401


##########
daffodil-cli/src/it/scala/org/apache/daffodil/CLI/Util.scala:
##########
@@ -384,6 +385,7 @@ object Util {
 
     def expect(matcher: Matcher[_]): Result = expect.expect(matcher)
     def expect(string: String): Result = expect.expect(contains(string))
+    def expectEOF(): Result = expect.expect(eof())

Review Comment:
   Still should revert this 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] jadams-tresys commented on a diff in pull request #878: Implement a encode/decodeEXI command to the CLI

Posted by GitBox <gi...@apache.org>.
jadams-tresys commented on code in PR #878:
URL: https://github.com/apache/daffodil/pull/878#discussion_r1034183604


##########
daffodil-cli/src/it/scala/org/apache/daffodil/CLI/Util.scala:
##########
@@ -384,6 +385,7 @@ object Util {
 
     def expect(matcher: Matcher[_]): Result = expect.expect(matcher)
     def expect(string: String): Result = expect.expect(contains(string))
+    def expectEOF(): Result = expect.expect(eof())

Review Comment:
   Sorry I missed this comment.  done



-- 
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] jadams-tresys commented on a diff in pull request #878: Implement a encode/decodeEXI command to the CLI

Posted by GitBox <gi...@apache.org>.
jadams-tresys commented on code in PR #878:
URL: https://github.com/apache/daffodil/pull/878#discussion_r1030424043


##########
daffodil-cli/src/it/scala/org/apache/daffodil/exi/TestEXIEncodeDecode.scala:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.saving
+
+import org.junit.Test
+import scala.xml.{Node, XML}
+
+import org.apache.daffodil.CLI.Util._
+import org.apache.daffodil.Main.ExitCode
+import org.apache.daffodil.xml.XMLUtils
+
+class TestCLIEncodeDecodeEXI {
+
+  @Test def test_3017_CLI_Encode_Decode_EXI_SA(): Unit = {
+    val schema = path("daffodil-test/src/test/resources/org/apache/daffodil/section06/namespaces/multi_base_15.dfdl.xsd")
+
+    withTempFile { tempEXI =>
+
+      // Create initial XML infoset
+      var infosetXML: Node = null
+      var encodedMD5: String = null
+      runCLI(args"parse -s $schema") { cli =>
+        cli.send("test", inputDone = true)
+        val res = cli.expectEOF
+        infosetXML = XML.loadString(res.getBefore)
+      } (ExitCode.Success)
+
+      // Encode infoset to schema aware EXI
+      runCLI(args"encodeEXI -s $schema -o $tempEXI") { cli =>
+        cli.sendLine(infosetXML.toString, inputDone = true)
+        encodedMD5 = md5sum(tempEXI)
+      } (ExitCode.Success)
+
+      // Decode EXI to XML and compare against original XML infoset
+      runCLI(args"decodeEXI -s $schema $tempEXI") { cli =>
+        val res = cli.expectEOF
+        val resultNode = XML.loadString(res.getBefore)

Review Comment:
   Resolved simply by using the withTempDirs function and specifying separate output files when running daffodil with the -o option.



-- 
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 #878: Implement a encode/decodeEXI command to the CLI

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on code in PR #878:
URL: https://github.com/apache/daffodil/pull/878#discussion_r1033961185


##########
daffodil-cli/src/main/scala/org/apache/daffodil/Main.scala:
##########
@@ -1236,8 +1266,77 @@ object Main {
         }
       }
 
-      // Required to avoid "match may not be exhaustive", but should never happen
-      case _ => Assert.impossible()
+      case Some(conf.exi) => {
+        var rc = ExitCode.Success
+        val exiOpts = conf.exi
+        val channel = exiOpts.output.toOption match {
+          case Some("-") | None => Channels.newChannel(STDOUT)
+          case Some(file) => new FileOutputStream(file).getChannel()
+        }
+        val output = Channels.newOutputStream(channel)
+
+        val inputStream = exiOpts.infile.toOption match {
+          case Some("-") | None => STDIN
+          case Some(file) => {
+            val f = new File(file)
+            new FileInputStream(f)
+          }
+        }
+        val input = new InputSource(inputStream)
+
+        val exiFactory = DefaultEXIFactory.newInstance
+        if (exiOpts.schema.isDefined) try {
+          val grammarFactory = GrammarFactory.newInstance
+          val grammar = grammarFactory.createGrammars(exiOpts.schema.toOption.get.toString, DFDLCatalogResolver.get)
+          exiFactory.setGrammars(grammar)
+        } catch {
+          case e: EXIException => {
+            Logger.log.error(s"Error creating EXI grammar for the supplied schema: ${ Misc.getSomeMessage(e).get }")
+            rc = ExitCode.Failure
+          }
+        }
+
+        (exiOpts.decode.toOption.get, rc) match {
+          case (true, ExitCode.Success) => { // Decoding
+            val exiSource = new EXISource(exiFactory)
+            exiSource.setInputSource(input)
+
+            val result = new StreamResult(output)
+            val tf = TransformerFactory.newInstance()
+            val transformer = tf.newTransformer
+            try {
+              transformer.transform(exiSource, result)
+            } catch {
+              case e: Exception => {
+                Logger.log.error(s"Error decoding EXI input: ${ Misc.getSomeMessage(e).get }")
+                rc = ExitCode.Failure
+              }
+            }
+          }
+          case (false, ExitCode.Success) => { // Encoding
+            val exiResult = new EXIResult(exiFactory)
+            exiResult.setOutputStream(output)
+
+            val reader = XMLReaderFactory.createXMLReader()
+            reader.setContentHandler(exiResult.getHandler)
+            try {
+              reader.parse(input)
+            } catch {
+              case s: org.xml.sax.SAXException => {
+                Logger.log.error(s"Error parsing input XML: ${ Misc.getSomeMessage(s).get }")
+                rc = ExitCode.Failure
+              }
+            }
+            output.close
+          }
+          case (_, _) => // Hit an exception creating exiFactory, rc already set
+        }
+
+        rc
+      }
+
+        // Required to avoid "match may not be exhaustive", but should never happen
+        case _ => Assert.impossible()

Review Comment:
   I think the indentation is off with this case?



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

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 #878: Implement a encode/decodeEXI command to the CLI

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on code in PR #878:
URL: https://github.com/apache/daffodil/pull/878#discussion_r1033959326


##########
daffodil-cli/src/main/scala/org/apache/daffodil/Main.scala:
##########
@@ -1236,8 +1266,77 @@ object Main {
         }
       }
 
-      // Required to avoid "match may not be exhaustive", but should never happen
-      case _ => Assert.impossible()
+      case Some(conf.exi) => {
+        var rc = ExitCode.Success
+        val exiOpts = conf.exi
+        val channel = exiOpts.output.toOption match {
+          case Some("-") | None => Channels.newChannel(STDOUT)
+          case Some(file) => new FileOutputStream(file).getChannel()
+        }
+        val output = Channels.newOutputStream(channel)
+
+        val inputStream = exiOpts.infile.toOption match {
+          case Some("-") | None => STDIN
+          case Some(file) => {
+            val f = new File(file)
+            new FileInputStream(f)
+          }
+        }
+        val input = new InputSource(inputStream)
+
+        val exiFactory = DefaultEXIFactory.newInstance
+        if (exiOpts.schema.isDefined) try {
+          val grammarFactory = GrammarFactory.newInstance
+          val grammar = grammarFactory.createGrammars(exiOpts.schema.toOption.get.toString, DFDLCatalogResolver.get)
+          exiFactory.setGrammars(grammar)

Review Comment:
   It's only a few lines, but I'd like to avoid this duplication of this similar logic in the InfosetHandler. Can we put this logic in a function that can be shared, maybe in he `EXIInfoseHandler` or some other object? E.g.
   
   ```scala
   object EXIInfosetHandler {
     ...
     def createEXIFactory(optSchema: Option[URI]): EXIFactory = {
       val exiFactory = DefaultEXIFactory.newInstance
       if (optSchema.isDefined)
         val grammarFactory = GrammarFactory.newInstance
         val grammar = grammarFactory.createGrammars(optSchema.get.toString, DFDLCatalogResolver.get)
         exiFactory.setGrammars(grammar)
       }
       exiFactory
     } 
   }
   ```
   
   Then the `apply` methods in the `EXIInfosetHandler` look something like
   
   ```scala
     /** non-schema aware EXI **/
     def apply(dataProcessor: DataProcessor): InfosetHandler = {
       EXIInfosetHandler(dataProcessor, createExiFactory(None))
     }
   
     /** schema aware EXI **/
     def apply(dataProcessor: DataProcessor, schemaUri: URI): InfosetHandler = {
       EXIInfosetHandler(dataProcessor, createExiFactory(Some(schemaUri))
     }
   ```
   
   And then this new code becomes something like this:
   
   ```scala
   val exiFactory = try {
     EXIInfosetHandler.createEXIFactory(exiOpts.schema.toOption)
   } catch {
      ...
   }
   ```
   
   This ensures we only have a single place where we deal with EXI factory creation logic.



-- 
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 #878: Implement a encode/decodeEXI command to the CLI

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on code in PR #878:
URL: https://github.com/apache/daffodil/pull/878#discussion_r1033960927


##########
daffodil-cli/src/main/scala/org/apache/daffodil/Main.scala:
##########
@@ -1236,8 +1266,77 @@ object Main {
         }
       }
 
-      // Required to avoid "match may not be exhaustive", but should never happen
-      case _ => Assert.impossible()
+      case Some(conf.exi) => {
+        var rc = ExitCode.Success
+        val exiOpts = conf.exi
+        val channel = exiOpts.output.toOption match {
+          case Some("-") | None => Channels.newChannel(STDOUT)
+          case Some(file) => new FileOutputStream(file).getChannel()
+        }
+        val output = Channels.newOutputStream(channel)
+
+        val inputStream = exiOpts.infile.toOption match {
+          case Some("-") | None => STDIN
+          case Some(file) => {
+            val f = new File(file)
+            new FileInputStream(f)
+          }
+        }
+        val input = new InputSource(inputStream)
+
+        val exiFactory = DefaultEXIFactory.newInstance
+        if (exiOpts.schema.isDefined) try {
+          val grammarFactory = GrammarFactory.newInstance
+          val grammar = grammarFactory.createGrammars(exiOpts.schema.toOption.get.toString, DFDLCatalogResolver.get)
+          exiFactory.setGrammars(grammar)
+        } catch {
+          case e: EXIException => {
+            Logger.log.error(s"Error creating EXI grammar for the supplied schema: ${ Misc.getSomeMessage(e).get }")
+            rc = ExitCode.Failure
+          }
+        }
+
+        (exiOpts.decode.toOption.get, rc) match {
+          case (true, ExitCode.Success) => { // Decoding
+            val exiSource = new EXISource(exiFactory)
+            exiSource.setInputSource(input)
+
+            val result = new StreamResult(output)
+            val tf = TransformerFactory.newInstance()
+            val transformer = tf.newTransformer
+            try {
+              transformer.transform(exiSource, result)
+            } catch {
+              case e: Exception => {

Review Comment:
   Can you add a comment why we catch a generic Exception? We usually try to avoid that except in cases where there are no other reasonable exceptions to catch, in which case it's worth a comment.



-- 
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 #878: Implement a encode/decodeEXI command to the CLI

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on code in PR #878:
URL: https://github.com/apache/daffodil/pull/878#discussion_r1033685351


##########
daffodil-cli/src/main/scala/org/apache/daffodil/Main.scala:
##########
@@ -31,7 +31,7 @@ import java.nio.charset.StandardCharsets
 import java.util.Scanner
 import java.util.concurrent.Executors
 import javax.xml.parsers.DocumentBuilderFactory
-import javax.xml.transform.TransformerFactory
+import javax.xml.transform.{ TransformerFactory, TransformerException }

Review Comment:
   Can you split these to two lines? I think we've started moving away from multiple imports on a single line.



##########
daffodil-cli/src/it/scala/org/apache/daffodil/exi/TestEXIEncodeDecode.scala:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.saving
+
+import org.junit.Test
+import scala.xml.XML
+import java.nio.file.Paths
+
+import org.apache.daffodil.CLI.Util._
+import org.apache.daffodil.Main.ExitCode
+import org.apache.daffodil.xml.XMLUtils
+
+class TestCLIEncodeDecodeEXI {
+
+  @Test def test_3017_CLI_Encode_Decode_EXI_SA(): Unit = {
+    val schema = path("daffodil-test/src/test/resources/org/apache/daffodil/usertests/Book2.dfdl.xsd")
+    val infosetPath = path("daffodil-test/src/test/resources/org/apache/daffodil/usertests/test_Book2.expected.xml")
+    val infosetXML = XML.loadFile(infosetPath.toFile)
+
+    withTempDir { tempDir =>
+      val tempEXI = Paths.get(tempDir.toString, "temp.exi")
+      val tempXML = Paths.get(tempDir.toString, "temp.xml")
+
+      // Encode infoset to schema aware EXI
+      runCLI(args"exi -s $schema -o $tempEXI") { cli =>
+        cli.sendLine(infosetXML.toString, inputDone = true)
+      } (ExitCode.Success)
+
+      // Decode EXI to XML and compare against original XML infoset
+      runCLI(args"exi -d -s $schema -o $tempXML $tempEXI") { cli =>
+      } (ExitCode.Success)
+
+      val resultNode = XML.loadFile(tempXML.toFile)
+      XMLUtils.compareAndReport(infosetXML, resultNode)
+    }
+  }
+
+  @Test def test_3017_CLI_Encode_Decode_EXI(): Unit = {
+    val inputXML = <person><name>Edward</name><age>42</age></person>
+
+    withTempDir { tempDir =>
+      val tempEXI = Paths.get(tempDir.toString, "temp.exi")
+      val tempXML = Paths.get(tempDir.toString, "temp.xml")
+
+      runCLI(args"exi -o $tempEXI") { cli =>
+        cli.sendLine(inputXML.toString, inputDone = true)
+      } (ExitCode.Success)
+
+      runCLI(args"exi -d -o $tempXML $tempEXI") { cli =>
+      } (ExitCode.Success)
+
+      val resultNode = XML.loadFile(tempXML.toFile)
+      XMLUtils.compareAndReport(inputXML, resultNode)
+    }
+  }
+
+  @Test def test_3017_CLI_EncodeBadFile_EXI(): Unit = {
+    val badXML = path("daffodil-test/src/test/resources/org/apache/daffodil/usertests/Book2.csv")
+
+    runCLI(args"exi $badXML") { cli =>
+      cli.expectErr("Error parsing input XML")
+    } (ExitCode.Failure)
+  }
+
+  @Test def test_3017_CLI_DecodeBadFile_EXI(): Unit = {
+    val badEXI = path("daffodil-test/src/test/resources/org/apache/daffodil/usertests/Book2.csv")
+
+    runCLI(args"exi -d $badEXI") { cli =>
+      cli.expectErr("No valid EXI document")
+    } (ExitCode.Failure)
+  }
+
+  @Test def test_3017_CLI_LoadBadSchema_EXI(): Unit = {
+    val badSchema = path("daffodil-test/src/test/resources/org/apache/daffodil/usertests/Book2.csv")
+
+    runCLI(args"exi -s $badSchema") { cli =>
+      cli.expectErr("Error creating EXI grammar for the supplied schema")
+    } (ExitCode.Failure)
+  }
+}

Review Comment:
   Thoughts on two additional tests/checks:
   1. Show that it errors if you try to encode an XML file with the `-s` option, but the XML doesn't match the provided schema
   1. Show that it errors if you try to decode a schema aware EXI file, but either no `-s` option was given or it was but it doesn't match what the EXI file was encoded with



##########
daffodil-cli/src/main/scala/org/apache/daffodil/Main.scala:
##########
@@ -514,12 +517,30 @@ class CLIConf(arguments: Array[String]) extends scallop.ScallopConf(arguments) {
     requireSubcommand()
   }
 
+  // Encode or decode EXI Subcommand Options
+  object exi extends scallop.Subcommand("exi") {
+    banner("""|Usage: daffodil exi [-d] [-s <schema>] [-o <output>] [infile]
+              |
+              |Encode/decode an XML file with EXI. If a schema is specified, it will use schema aware encoding/decoding.
+              |
+              |EncodeEXI Options:""".stripMargin)
+
+    descr("Encode an XML file with EXI")
+    helpWidth(width)
+
+    val output = opt[String](argName = "file", descr = "Output file to write the encoded/decoded file to.")
+    val schema = opt[URI]("schema", argName = "file", descr = "DFDL schema to use for schema aware encoding/decoding.")(fileResourceURIConverter)
+    val infile = trailArg[String](required = false, descr = "Input XML file to encode. If not specified, or a value of -, reads from stdin.")

Review Comment:
   Can we move infile to the end of the options (i.e. after decode) like we do with the other subcommands? Makes it more clear that it's a trailing argument.



##########
daffodil-cli/src/main/scala/org/apache/daffodil/Main.scala:
##########
@@ -1482,6 +1503,78 @@ object Main {
         }
       }
 
+      case Some(conf.exi) => {
+        val exiOpts = conf.exi
+        val channel = exiOpts.output.toOption match {
+          case Some("-") | None => Channels.newChannel(STDOUT)
+          case Some(file) => new FileOutputStream(file).getChannel()
+        }
+        val output = Channels.newOutputStream(channel)
+
+        val inputStream = exiOpts.infile.toOption match {
+          case Some("-") | None => STDIN
+          case Some(file) => {
+            val f = new File(file)
+            new FileInputStream(f)
+          }
+        }
+        val input = new InputSource(inputStream)
+
+        val exiFactory = try {
+          if (exiOpts.schema.isDefined)
+            getExiFactoryOpt(InfosetType.EXISA, exiOpts.schema.toOption)
+          else
+            getExiFactoryOpt(InfosetType.EXI, exiOpts.schema.toOption)
+        } catch {
+          case e: EXIException => {
+            Logger.log.error("Error creating EXI grammar for the supplied schema: %s".format(Misc.getSomeMessage(e).get))
+            return ExitCode.Failure
+          }
+        }
+
+        if (exiOpts.decode.toOption.get) { // Decoding EXI file to XML
+          val exiSource = new EXISource(exiFactory.get)
+          exiSource.setInputSource(input)
+
+          val result = new StreamResult(output)
+          val tf = TransformerFactory.newInstance()
+          val transformer = tf.newTransformer
+          try {
+            transformer.transform(exiSource, result)
+          } catch {
+            case t: TransformerException => {
+              Logger.log.error("Error decoding EXI input: %s".format(Misc.getSomeMessage(t).get))

Review Comment:
   We've been trying to use the `s` string interpolator, it avoids issues where the format options don't match the format string. So instead this could be something like this:
   ```scala
   Logger.log.error(s"Error decoding EXI input: ${ Misc.getSomeMessage(t) }")
   ```
   Same with other log messages below.



##########
daffodil-cli/src/main/scala/org/apache/daffodil/Main.scala:
##########
@@ -1482,6 +1503,78 @@ object Main {
         }
       }
 
+      case Some(conf.exi) => {
+        val exiOpts = conf.exi
+        val channel = exiOpts.output.toOption match {
+          case Some("-") | None => Channels.newChannel(STDOUT)
+          case Some(file) => new FileOutputStream(file).getChannel()
+        }
+        val output = Channels.newOutputStream(channel)
+
+        val inputStream = exiOpts.infile.toOption match {
+          case Some("-") | None => STDIN
+          case Some(file) => {
+            val f = new File(file)
+            new FileInputStream(f)
+          }
+        }
+        val input = new InputSource(inputStream)
+
+        val exiFactory = try {
+          if (exiOpts.schema.isDefined)
+            getExiFactoryOpt(InfosetType.EXISA, exiOpts.schema.toOption)
+          else
+            getExiFactoryOpt(InfosetType.EXI, exiOpts.schema.toOption)
+        } catch {
+          case e: EXIException => {
+            Logger.log.error("Error creating EXI grammar for the supplied schema: %s".format(Misc.getSomeMessage(e).get))
+            return ExitCode.Failure

Review Comment:
   It might make things a bit more complicated, but we should avoid the use of `return`. Same with other uses of return below.



##########
daffodil-cli/src/main/scala/org/apache/daffodil/Main.scala:
##########
@@ -514,12 +517,30 @@ class CLIConf(arguments: Array[String]) extends scallop.ScallopConf(arguments) {
     requireSubcommand()
   }
 
+  // Encode or decode EXI Subcommand Options
+  object exi extends scallop.Subcommand("exi") {
+    banner("""|Usage: daffodil exi [-d] [-s <schema>] [-o <output>] [infile]
+              |
+              |Encode/decode an XML file with EXI. If a schema is specified, it will use schema aware encoding/decoding.
+              |
+              |EncodeEXI Options:""".stripMargin)
+
+    descr("Encode an XML file with EXI")
+    helpWidth(width)
+
+    val output = opt[String](argName = "file", descr = "Output file to write the encoded/decoded file to.")

Review Comment:
   In our other `output` options we add `If not given or is -, infoset is written to stdout.` I believe this has the same behavior.



##########
daffodil-cli/src/main/scala/org/apache/daffodil/Main.scala:
##########
@@ -1482,6 +1503,78 @@ object Main {
         }
       }
 
+      case Some(conf.exi) => {
+        val exiOpts = conf.exi
+        val channel = exiOpts.output.toOption match {
+          case Some("-") | None => Channels.newChannel(STDOUT)
+          case Some(file) => new FileOutputStream(file).getChannel()
+        }
+        val output = Channels.newOutputStream(channel)
+
+        val inputStream = exiOpts.infile.toOption match {
+          case Some("-") | None => STDIN
+          case Some(file) => {
+            val f = new File(file)
+            new FileInputStream(f)
+          }
+        }
+        val input = new InputSource(inputStream)
+
+        val exiFactory = try {
+          if (exiOpts.schema.isDefined)
+            getExiFactoryOpt(InfosetType.EXISA, exiOpts.schema.toOption)
+          else
+            getExiFactoryOpt(InfosetType.EXI, exiOpts.schema.toOption)
+        } catch {
+          case e: EXIException => {
+            Logger.log.error("Error creating EXI grammar for the supplied schema: %s".format(Misc.getSomeMessage(e).get))
+            return ExitCode.Failure
+          }
+        }
+
+        if (exiOpts.decode.toOption.get) { // Decoding EXI file to XML
+          val exiSource = new EXISource(exiFactory.get)
+          exiSource.setInputSource(input)
+
+          val result = new StreamResult(output)
+          val tf = TransformerFactory.newInstance()
+          val transformer = tf.newTransformer
+          try {
+            transformer.transform(exiSource, result)
+          } catch {
+            case t: TransformerException => {
+              Logger.log.error("Error decoding EXI input: %s".format(Misc.getSomeMessage(t).get))
+              return ExitCode.Failure
+            }
+            case s: org.xml.sax.SAXException => {
+              Logger.log.error("Error decoding EXI input: %s".format(Misc.getSomeMessage(s).get))
+              return ExitCode.Failure
+            }
+            case e: EXIException => {
+              Logger.log.error("Error decoding EXI input: %s".format(Misc.getSomeMessage(e).get))
+              return ExitCode.Failure

Review Comment:
   It would be nice if we could have tests that hit these errors. I suggested some that might cover some of these.



-- 
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] jadams-tresys commented on a diff in pull request #878: Implement a encode/decodeEXI command to the CLI

Posted by GitBox <gi...@apache.org>.
jadams-tresys commented on code in PR #878:
URL: https://github.com/apache/daffodil/pull/878#discussion_r1030422907


##########
daffodil-cli/src/it/scala/org/apache/daffodil/exi/TestEXIEncodeDecode.scala:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.saving
+
+import org.junit.Test
+import scala.xml.{Node, XML}
+
+import org.apache.daffodil.CLI.Util._
+import org.apache.daffodil.Main.ExitCode
+import org.apache.daffodil.xml.XMLUtils
+
+class TestCLIEncodeDecodeEXI {
+
+  @Test def test_3017_CLI_Encode_Decode_EXI_SA(): Unit = {
+    val schema = path("daffodil-test/src/test/resources/org/apache/daffodil/section06/namespaces/multi_base_15.dfdl.xsd")
+
+    withTempFile { tempEXI =>
+
+      // Create initial XML infoset
+      var infosetXML: Node = null
+      var encodedMD5: String = null
+      runCLI(args"parse -s $schema") { cli =>
+        cli.send("test", inputDone = true)
+        val res = cli.expectEOF
+        infosetXML = XML.loadString(res.getBefore)
+      } (ExitCode.Success)

Review Comment:
   Thought it might be a useful test to demonstrate that the EXI infoset generated by parsing directly into EXI would be the same as encoding an XML infoset into EXI.



-- 
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 #878: Implement a encode/decodeEXI command to the CLI

Posted by GitBox <gi...@apache.org>.
tuxji commented on code in PR #878:
URL: https://github.com/apache/daffodil/pull/878#discussion_r1026907624


##########
daffodil-cli/src/main/scala/org/apache/daffodil/Main.scala:
##########
@@ -1482,6 +1518,74 @@ object Main {
         }
       }
 
+      case Some(conf.encodeEXI) => {
+        val encodeOpts = conf.encodeEXI

Review Comment:
   I'm puzzled why we're seeing codecov check warnings that some lines are not covered by tests, when we do have a new test.  



##########
daffodil-cli/src/main/scala/org/apache/daffodil/Main.scala:
##########
@@ -1482,6 +1518,74 @@ object Main {
         }
       }
 
+      case Some(conf.encodeEXI) => {
+        val encodeOpts = conf.encodeEXI
+        val channel = encodeOpts.output.toOption match {
+          case Some("-") | None => Channels.newChannel(STDOUT)
+          case Some(file) => new FileOutputStream(file).getChannel()
+        }
+        val output = Channels.newOutputStream(channel)
+
+        val inputStream = encodeOpts.infile.toOption match {
+          case Some("-") | None => STDIN
+          case Some(file) => {
+            val f = new File(file)
+            new FileInputStream(f)
+          }
+        }
+        val input = new InputSource(inputStream)
+
+        val exiFactory = {
+          if (encodeOpts.schema.isDefined)
+            getExiFactoryOpt(InfosetType.EXISA, encodeOpts.schema.toOption)
+          else
+            getExiFactoryOpt(InfosetType.EXI, encodeOpts.schema.toOption)
+        }
+        val exiResult = new EXIResult(exiFactory.get)
+        exiResult.setOutputStream(output)
+
+        val reader = XMLReaderFactory.createXMLReader()
+        reader.setContentHandler(exiResult.getHandler)
+        reader.parse(input)
+        output.close

Review Comment:
   I would like to see tests which encode an invalid XML file and decode an invalid EXI file to make sure they get an error message, not a bug stack trace.  I'm sure people will try to encode an EXI file and decode an XML file the wrong way around and they should see an appropriate error message.



##########
daffodil-cli/src/it/scala/org/apache/daffodil/CLI/Util.scala:
##########
@@ -384,6 +385,7 @@ object Util {
 
     def expect(matcher: Matcher[_]): Result = expect.expect(matcher)
     def expect(string: String): Result = expect.expect(contains(string))
+    def expectEOF(): Result = expect.expect(eof())

Review Comment:
   If you decide to stop calling expectEOF and getBefore and write to a temp file instead as Steve suggested, then you'll want to revert your changes here.



##########
daffodil-cli/src/it/scala/org/apache/daffodil/exi/TestEXIEncodeDecode.scala:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.saving
+
+import org.junit.Test
+import scala.xml.{Node, XML}
+
+import org.apache.daffodil.CLI.Util._
+import org.apache.daffodil.Main.ExitCode
+import org.apache.daffodil.xml.XMLUtils
+
+class TestCLIEncodeDecodeEXI {
+
+  @Test def test_3017_CLI_Encode_Decode_EXI_SA(): Unit = {
+    val schema = path("daffodil-test/src/test/resources/org/apache/daffodil/section06/namespaces/multi_base_15.dfdl.xsd")
+
+    withTempFile { tempEXI =>
+
+      // Create initial XML infoset
+      var infosetXML: Node = null
+      var encodedMD5: String = null
+      runCLI(args"parse -s $schema") { cli =>
+        cli.send("test", inputDone = true)
+        val res = cli.expectEOF
+        infosetXML = XML.loadString(res.getBefore)
+      } (ExitCode.Success)
+
+      // Encode infoset to schema aware EXI
+      runCLI(args"encodeEXI -s $schema -o $tempEXI") { cli =>
+        cli.sendLine(infosetXML.toString, inputDone = true)
+        encodedMD5 = md5sum(tempEXI)
+      } (ExitCode.Success)
+
+      // Decode EXI to XML and compare against original XML infoset
+      runCLI(args"decodeEXI -s $schema $tempEXI") { cli =>
+        val res = cli.expectEOF
+        val resultNode = XML.loadString(res.getBefore)

Review Comment:
   Once you write to a tempFile, you can use commons-io to compare two temp files by calling [FileUtils.contentsEquals](https://commons.apache.org/proper/commons-io/javadocs/api-2.5/org/apache/commons/io/FileUtils.html#contentEquals(java.io.File,%20java.io.File)).



##########
daffodil-cli/src/main/scala/org/apache/daffodil/Main.scala:
##########
@@ -514,12 +516,46 @@ class CLIConf(arguments: Array[String]) extends scallop.ScallopConf(arguments) {
     requireSubcommand()
   }
 
+  // Encode EXI Subcommand Options
+  object encodeEXI extends scallop.Subcommand("encodeEXI") {
+    banner("""|Usage: daffodil encodeEXI [-s <schema>] [-o <output>] [infile]

Review Comment:
   Yes, I think it's better to have a single subcommand `exi` which normally encodes XML to EXI and optionally decodes EXI to XML with the `-d` or `--decode` options.



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

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 pull request #878: Implement a encode/decodeEXI command to the CLI

Posted by GitBox <gi...@apache.org>.
tuxji commented on PR #878:
URL: https://github.com/apache/daffodil/pull/878#issuecomment-1320566001

   > I'm sure people will try to encode an EXI file and decode an XML file the wrong way around and they should see an appropriate error message.
   
   I remember Mike commented in the JIRA issue that XML and EXI files look so different that we might not even need a `--decode` option.  It may be more work to implement but it might be nice to have file type recognition rather than `-d`.


-- 
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 #878: Implement a encode/decodeEXI command to the CLI

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on code in PR #878:
URL: https://github.com/apache/daffodil/pull/878#discussion_r1033964161


##########
daffodil-cli/src/it/scala/org/apache/daffodil/CLI/Util.scala:
##########
@@ -384,6 +385,7 @@ object Util {
 
     def expect(matcher: Matcher[_]): Result = expect.expect(matcher)
     def expect(string: String): Result = expect.expect(contains(string))
+    def expectEOF(): Result = expect.expect(eof())

Review Comment:
   Agreed, I think expecting an EOF is usually going to lead to issues, so should be avoided in most cases. If it's not part of the CLI integration "API", it makes it less likely to be used, but it's still possible via `expect.expect(eof)` if it's really needed.



-- 
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] jadams-tresys commented on pull request #878: Implement a encode/decodeEXI command to the CLI

Posted by GitBox <gi...@apache.org>.
jadams-tresys commented on PR #878:
URL: https://github.com/apache/daffodil/pull/878#issuecomment-1331009938

   All issues should be addressed, including errors that were being seen on Windows CI machines. @tuxji just need your approval as I removed the expectEOF 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