You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by "mbeckerle (via GitHub)" <gi...@apache.org> on 2023/10/14 03:12:56 UTC

[PR] WIP: Preliminary Review on adding Daffodil to Drill (drill)

mbeckerle opened a new pull request, #2836:
URL: https://github.com/apache/drill/pull/2836

   # [DRILL-2835](https://issues.apache.org/jira/browse/DRILL-2835): Preliminary Review on adding Daffodil to Drill
   
   ## Description
   
   New format-daffodil module created. But I need assistance with several aspects. 
   
   Tests of creating Drill schemas from DFDL working. They're simple, but it's showing promise.
   
   There are major TODO/FIXME/TBDs in here. Search for FIXME, and "Then a MIRACLE occurs..."
   
   This does not compile yet because of the plugin system and how to initialize things. This is the main open problem to get it to compile without error.
   
   Needs review by Drill-devs.
   
   ## Documentation
   
   TBD: This will require doc eventually
   
   ## Testing
   
   Needs more. This is just a preliminary design review Work-in-progress.
   
   


-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] WIP: Preliminary Review on adding Daffodil to Drill (drill)

Posted by "jnturton (via GitHub)" <gi...@apache.org>.
jnturton commented on code in PR #2836:
URL: https://github.com/apache/drill/pull/2836#discussion_r1365726721


##########
contrib/pom.xml:
##########
@@ -59,6 +59,7 @@
     <module>format-pcapng</module>
     <module>format-iceberg</module>
     <module>format-deltalake</module>
+    <module>format-daffodil</module>

Review Comment:
   @mbeckerle we're starting to sort such lists as @cgivre says, but only in #2825 which hasn't been merged because I've been dragging my feet. Let me get it merged then you can rebase your PR and you'll find a sorted list to add to.



-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] WIP: Preliminary Review on adding Daffodil to Drill (drill)

Posted by "jnturton (via GitHub)" <gi...@apache.org>.
jnturton commented on code in PR #2836:
URL: https://github.com/apache/drill/pull/2836#discussion_r1367675681


##########
contrib/pom.xml:
##########
@@ -59,6 +59,7 @@
     <module>format-pcapng</module>
     <module>format-iceberg</module>
     <module>format-deltalake</module>
+    <module>format-daffodil</module>

Review Comment:
   @mbeckerle #2825 is merged, please rebase on master when convenient.



-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] DRILL-8474: Add Daffodil Format Plugin (drill)

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


##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilBatchReader.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.drill.exec.store.daffodil;
+
+import org.apache.daffodil.japi.DataProcessor;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.physical.impl.scan.v3.ManagedReader;
+import org.apache.drill.exec.physical.impl.scan.v3.file.FileDescrip;
+import org.apache.drill.exec.physical.impl.scan.v3.file.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.store.daffodil.schema.DaffodilDataProcessorFactory;
+import org.apache.drill.exec.store.dfs.DrillFileSystem;
+import org.apache.drill.exec.store.dfs.easy.EasySubScan;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Objects;
+
+import static org.apache.drill.exec.store.daffodil.schema.DaffodilDataProcessorFactory.*;
+import static org.apache.drill.exec.store.daffodil.schema.DrillDaffodilSchemaUtils.daffodilDataProcessorToDrillSchema;
+
+public class DaffodilBatchReader implements ManagedReader {
+
+  private static final Logger logger = LoggerFactory.getLogger(DaffodilBatchReader.class);
+  private final RowSetLoader rowSetLoader;
+  private final CustomErrorContext errorContext;
+  private final DaffodilMessageParser dafParser;
+  private final InputStream dataInputStream;
+
+  public DaffodilBatchReader(DaffodilReaderConfig readerConfig, EasySubScan scan,
+      FileSchemaNegotiator negotiator) {
+
+    errorContext = negotiator.parentErrorContext();
+    DaffodilFormatConfig dafConfig = readerConfig.plugin.getConfig();
+
+    String schemaURIString = dafConfig.getSchemaURI(); // "schema/complexArray1.dfdl.xsd";
+    String rootName = dafConfig.getRootName();
+    String rootNamespace = dafConfig.getRootNamespace();
+    boolean validationMode = dafConfig.getValidationMode();
+
+    URI dfdlSchemaURI;
+    try {
+      dfdlSchemaURI = new URI(schemaURIString);
+    } catch (URISyntaxException e) {
+      throw UserException.validationError(e).build(logger);
+    }
+
+    FileDescrip file = negotiator.file();
+    DrillFileSystem fs = file.fileSystem();
+    URI fsSchemaURI = fs.getUri().resolve(dfdlSchemaURI);
+
+    DaffodilDataProcessorFactory dpf = new DaffodilDataProcessorFactory();
+    DataProcessor dp;
+    try {
+      dp = dpf.getDataProcessor(fsSchemaURI, validationMode, rootName, rootNamespace);
+    } catch (CompileFailure e) {
+      throw UserException.dataReadError(e)
+          .message(String.format("Failed to get Daffodil DFDL processor for: %s", fsSchemaURI))
+          .addContext(errorContext).addContext(e.getMessage()).build(logger);
+    }
+    // Create the corresponding Drill schema.
+    // Note: this could be a very large schema. Think of a large complex RDBMS schema,
+    // all of it, hundreds of tables, but all part of the same metadata tree.
+    TupleMetadata drillSchema = daffodilDataProcessorToDrillSchema(dp);
+    // Inform Drill about the schema
+    negotiator.tableSchema(drillSchema, true);
+
+    //
+    // DATA TIME: Next we construct the runtime objects, and open files.
+    //
+    // We get the DaffodilMessageParser, which is a stateful driver for daffodil that
+    // actually does the parsing.
+    rowSetLoader = negotiator.build().writer();
+
+    // We construct the Daffodil InfosetOutputter which the daffodil parser uses to
+    // convert infoset event calls to fill in a Drill row via a rowSetLoader.
+    DaffodilDrillInfosetOutputter outputter = new DaffodilDrillInfosetOutputter(rowSetLoader);
+
+    // Now we can set up the dafParser with the outputter it will drive with
+    // the parser-produced infoset.
+    dafParser = new DaffodilMessageParser(dp); // needs further initialization after this.
+    dafParser.setInfosetOutputter(outputter);
+
+    Path dataPath = file.split().getPath();
+    // Lastly, we open the data stream
+    try {
+      dataInputStream = fs.openPossiblyCompressedStream(dataPath);
+    } catch (IOException e) {
+      throw UserException.dataReadError(e)
+          .message(String.format("Failed to open input file: %s", dataPath.toString()))
+          .addContext(errorContext).addContext(e.getMessage()).build(logger);
+    }
+    // And lastly,... tell daffodil the input data stream.
+    dafParser.setInputStream(dataInputStream);
+  }
+
+  /**
+   * This is the core of actual processing - data movement from Daffodil to Drill.
+   * <p>
+   * If there is space in the batch, and there is data available to parse then this calls the
+   * daffodil parser, which parses data, delivering it to the rowWriter by way of the infoset
+   * outputter.
+   * <p>
+   * Repeats until the rowWriter is full (a batch is full), or there is no more data, or a parse
+   * error ends execution with a throw.
+   * <p>
+   * Validation errors and other warnings are not errors and are logged but do not cause parsing to
+   * fail/throw.
+   *
+   * @return true if there are rows retrieved, false if no rows were retrieved, which means no more
+   *     will ever be retrieved (end of data).
+   * @throws RuntimeException
+   *     on parse errors.
+   */
+  @Override
+  public boolean next() {
+    // Check assumed invariants
+    // We don't know if there is data or not. This could be called on an empty data file.
+    // We DO know that this won't be called if there is no space in the batch for even 1
+    // row.
+    if (dafParser.isEOF()) {
+      return false; // return without even checking for more rows or trying to parse.
+    }
+    while (rowSetLoader.start() && !dafParser.isEOF()) { // we never zero-trip this loop.
+      // the predicate is always true once.
+      dafParser.parse();
+      if (dafParser.isProcessingError()) {
+        assert (Objects.nonNull(dafParser.getDiagnostics()));
+        throw UserException.dataReadError().message(dafParser.getDiagnosticsAsString())
+            .addContext(errorContext).build(logger);
+      }
+      if (dafParser.isValidationError()) {
+        logger.warn(dafParser.getDiagnosticsAsString());

Review Comment:
   Agree. 
   
   We draw a distinction between "well formed" and "invalid" data and whether one does validation seems like the right switch in daffodil to use. 
   
   If data is malformed, that means you can't successfully parse it. If it is invalid, that just means values are unexpected. Example: A 3 digit number representing a percentage 0 to 100. -1 is invalid, ABC is malformed. 
   
   If data is not well formed, you really cannot continue parsing it, as you cannot convert it to the type expected. But, if you are able to determine at least how big it is, it's possible to capture that length of data into a dummy "badData" element which is always invalid (so isn't a "false positive" parse). This capability has to be designed into the DFDL schema, but it is something we've been doing more and more. 
   
   Hence, one can tolerate even _some_ malformed data. If it is malformed to where you cannot determine the length, then continuing is impossible. 
   
   We will see if more than this is needed. Options like the "use all strings/varchar" or all numbers are float, which you have for toleratng situations with other data connectors *may* prove useful, particularly while a DFDL schema is in development and you are really just testing it (and the corresponding data) using Drill. 



-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] DRILL-2835: Daffodil Feature for Drill (drill)

Posted by "mbeckerle (via GitHub)" <gi...@apache.org>.
mbeckerle commented on PR #2836:
URL: https://github.com/apache/drill/pull/2836#issuecomment-1874213780

   @cgivre yes, the next architectural-level issue is how to get a compiled DFDL schema out to everyplace Drill will run a Daffodil parse. Every one of those JVMs needs to reload it. 
   
   I'll do the various cleanups and such. The one issue I don't know how to fix is the "typed setter" vs. (set-object) issue, so if you could steer me in the right direction on that it would help. 


-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] WIP: Preliminary Review on adding Daffodil to Drill (drill)

Posted by "cgivre (via GitHub)" <gi...@apache.org>.
cgivre commented on PR #2836:
URL: https://github.com/apache/drill/pull/2836#issuecomment-1783992062

   @mbeckerle Looks like you're making good progress!  


-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] WIP: Preliminary Review on adding Daffodil to Drill (drill)

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


##########
contrib/format-daffodil/.gitignore:
##########


Review Comment:
   Remove this file from change set. 



##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilDrillInfosetOutputter.java:
##########
@@ -0,0 +1,180 @@
+/*
+ * 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.drill.exec.store.daffodil;
+
+import org.apache.daffodil.lib.xml.NamedQName;
+import org.apache.daffodil.runtime1.api.BlobMethodsImpl;
+import org.apache.daffodil.runtime1.api.ComplexElementMetadata;
+import org.apache.daffodil.runtime1.api.ElementMetadata;
+import org.apache.daffodil.runtime1.api.InfosetArray;
+import org.apache.daffodil.runtime1.api.InfosetComplexElement;
+import org.apache.daffodil.runtime1.api.InfosetOutputter;
+import org.apache.daffodil.runtime1.api.InfosetSimpleElement;
+import org.apache.daffodil.runtime1.api.Status;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.TupleSchema;
+import org.apache.drill.exec.vector.accessor.ArrayWriter;
+import org.apache.drill.exec.vector.accessor.ColumnWriter;
+import org.apache.drill.exec.vector.accessor.ObjectWriter;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.drill.exec.vector.accessor.writer.AbstractArrayWriter;
+import org.apache.drill.exec.vector.accessor.writer.AbstractTupleWriter;
+import org.apache.drill.exec.vector.accessor.writer.MapWriter;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Enumeration;
+
+import java.util.Stack;
+
+/**
+ * Adapts Daffodil parser infoset event calls to Drill writer calls
+ * to fill in Drill data rows.
+ */
+public class DaffodilDrillInfosetOutputter
+    extends BlobMethodsImpl implements InfosetOutputter {
+
+  private final Stack<ColumnWriter> columnWriterStack = new Stack<ColumnWriter>();
+
+  private ColumnWriter columnWriter() {
+    return columnWriterStack.peek();
+  }
+
+  private static final Logger logger = LoggerFactory.getLogger(DaffodilDrillInfosetOutputter.class);
+
+  private DaffodilDrillInfosetOutputter() {} // no default constructor
+
+  public DaffodilDrillInfosetOutputter(RowSetLoader writer) {
+    columnWriterStack.push(writer);
+  }
+
+  @Override
+  public void reset() {
+  }
+
+  @Override
+  public void startDocument() {}
+
+  @Override
+  public void endDocument() {}
+
+  private String colName(ElementMetadata erd) {
+    NamedQName nqn = erd.namedQName();
+    String colName = nqn.toPrefix_localName(); // just the local name if no prefix/namespace
+    return colName;
+  }
+
+  @Override
+  public void startSimple(InfosetSimpleElement diSimple) {
+    ElementMetadata erd = diSimple.metadata();
+    boolean isNilled = diSimple.isNilled();
+    String colName = colName(erd);
+    // If the column is an array, startArray will have setup columnWriter() to be the array writer.
+    // Otherwise, it's a TupleMetadata and we need to get this simple element's column.
+    // or it could be a top-level simple element (which we do use for test cases)
+    // in which case the columnWriter is the original RowSetWriter.
+    System.err.println(columnWriter().getClass().getName());
+    ColumnWriter cw = columnWriter();
+    switch (cw.type()) {
+      case ARRAY: {
+        assert(erd.isArray());
+        // do nothing startArray has this ready to write.'
+        break;
+      }
+      case TUPLE: {
+        cw = ((TupleWriter)cw).column(colName);
+        break;
+      }
+    }
+    ColumnMetadata cm = cw.schema();
+    if (isNilled) {
+      assert(cm.isNullable());
+      cw.setNull(); // Can I have a nullable array item. I.e., each item is nullable?
+    } else {
+      //
+      // FIXME: only INT is implemented right now.
+      //
+      int value = diSimple.getInt(); // will fail on downcast if not an INT.
+      cw.setObject(value); // autoboxing for primitive types.
+    }
+  }
+
+  @Override
+  public void endSimple(InfosetSimpleElement diSimple) {
+     /// nothing to do.
+  }
+
+  @Override
+  public void startComplex(InfosetComplexElement diComplex) {
+    ComplexElementMetadata erd = diComplex.metadata();
+    String colName = colName(erd);
+    ColumnWriter cw = columnWriter();
+    switch (cw.type()) {
+      case ARRAY: {
+        //

Review Comment:
   This issue is fixed, at least as far as a test reads 3 records each containing two fields. 



##########
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/SchemaNegotiator.java:
##########
@@ -78,7 +78,7 @@
  * Some readers can determine the source schema at the start of a scan.
  * For example, a CSV file has headers, a Parquet file has footers, both
  * of which define a schema. This case is called "early schema." The
- * reader fefines the schema by calling
+ * reader defines the schema by calling

Review Comment:
   Revert this. I need to stay focused in just my part of things. 



-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] DRILL-2835: Daffodil Feature for Drill (drill)

Posted by "paul-rogers (via GitHub)" <gi...@apache.org>.
paul-rogers commented on PR #2836:
URL: https://github.com/apache/drill/pull/2836#issuecomment-1874845274

   Hi Mike,
   
   Just jumping in with a random thought. Drill has accumulated a number of
   schema systems: Parquet metadata cache, HMS, Drill's own metastore,
   "provided schema", and now DFDL. All provide ways of defining data: be it
   Parquet, JSON, CSV or whatever. One can't help but wonder, should some
   future version try to reduce this variation somewhat? Maybe map all the
   variations to DFDL? Map DFDL to Drill's own mechanisms?
   
   Drill uses two kinds of metadata: schema definitions and file metadata used
   for scan pruning. Schema information could be used at plan time (to provide
   column types), but certainly at scan time (to "discover" the defined
   schema.) File metadata is used primarily at plan time to work out how to
   distribute work.
   
   A bit of background on scan pruning. Back in the day, it was common to have
   thousands or millions of files in Hadoop to scan: this was why tools like
   Drill were distributed: divide and conquer. And, of course, the fastest
   scan is to skip files that we know can't contain the information we want.
   File metadata captures this information outside of the files themselves.
   HMS was the standard solution in the Hadoop days. (Amazon Glue, for S3, is
   evidently based on HMS.)
   
   For example, Drill's Parquet metadata cache, the Drill metastore and HMS
   all provide both schema and file metadata information. The schema
   information mainly helped with schema evolution: over time, different files
   have different sets of columns. File metadata provides information *about*
   the file, such as the data ranges stored in each file. For Parquet, we
   might track that '2023-01-Boston.parquet' has data from the office='Boston'
   range. (So, no use scanning the file for office='Austin'.) And so on.
   
   With Hadoop HFS, it was customary to use directory structure as a partial
   primary index: our file above would live in the /sales/2023/01 directory,
   for example, and logic chooses the proper set of directories to scan. In
   Drill, it is up to the user to add crufty conditionals on the path name. In
   Impala, and other HMS-aware tools, the user just says WHERE order_year =
   2023 AND order_month = 1, and HMS tells the tool that the order_year and
   order_month columns translate to such-and-so directory paths. Would be nice
   if Drill could provide that feature as well, given the proper file
   metadata: in this case, the mapping of column names to path directories and
   file names.
   
   Does DFDL provide only schema information? Does it support versioning so
   that we know that "old.csv" lacks the "version" column, while "new.csv"
   includes that column? Does it also include the kinds of file metadata
   mentioned above?
   
   Or, perhaps DFDL is used in a different context in which the files have a
   fixed schema and are small in number? This would fit well the "desktop
   analytics" model that Charles and James suggested is where Drill is now
   most commonly used.
   
   The answers might suggest if DFDL can be the universal data description. or
   if DFDL applies just to individual file schemas, and Drill would still need
   a second system to track schema evolution and file metadata for large
   deployments.
   
   Further, if DFDL is kind of a stand-alone thing, with its own reader, then
   we end up with more complexity: the Drill JSON reader and the DFDL JSON
   reader. Same for CSV, etc. JSON is so complex that we'd find ourselves
   telling people that the quirks work one way with the native reader, another
   way with DFDL. Plus, the DFDL readers might not handle file splits the same
   way, or support the same set of formats that Drill's other readers support,
   and so on. It would be nice to separate the idea of schema description from
   reader implementation, so that DFDL can be used as a source of schema for
   any arbitrary reader: both at plan and scan times.
   
   If DFDL uses its own readers, then we'd need DFDL reader representations in
   Calcite, which would pick up DFDL schemas so that the schemas are reliably
   serialized out to each node as part of the physical plan. This is possible,
   but it does send us down the two-readers-for-every-format path.
   
   On the other hand, if DFDL mapped to Drill's existing schema description,
   then DFDL could be used with our existing readers and there would be just
   one schema description sent to readers: Drill's existing provided schema
   format that EVF can already consume. At present, just a few formats support
   provided schema in the Calcite layer: CSV for sure, maybe JSON?
   
   Any thoughts on where this kind of thing might evolve with DFDL in the
   picture?
   
   Thanks,
   
   - Paul
   
   
   On Tue, Jan 2, 2024 at 8:00 AM Mike Beckerle ***@***.***>
   wrote:
   
   > @cgivre <https://github.com/cgivre> yes, the next architectural-level
   > issue is how to get a compiled DFDL schema out to everyplace Drill will run
   > a Daffodil parse. Every one of those JVMs needs to reload it.
   >
   > I'll do the various cleanups and such. The one issue I don't know how to
   > fix is the "typed setter" vs. (set-object) issue, so if you could steer me
   > in the right direction on that it would help.
   >
   > —
   > Reply to this email directly, view it on GitHub
   > <https://github.com/apache/drill/pull/2836#issuecomment-1874213780>, or
   > unsubscribe
   > <https://github.com/notifications/unsubscribe-auth/AAYZF4MFVRCUYDCKJYSKKYTYMQVLFAVCNFSM6AAAAAA576F7J2VHI2DSMVQWIX3LMV43OSLTON2WKQ3PNVWWK3TUHMYTQNZUGIYTGNZYGA>
   > .
   > You are receiving this because you were mentioned.Message ID:
   > ***@***.***>
   >
   


-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] DRILL-8474: Add Daffodil Format Plugin (drill)

Posted by "cgivre (via GitHub)" <gi...@apache.org>.
cgivre commented on code in PR #2836:
URL: https://github.com/apache/drill/pull/2836#discussion_r1441799950


##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilDrillInfosetOutputter.java:
##########
@@ -0,0 +1,296 @@
+/*
+ * 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.drill.exec.store.daffodil;
+
+import org.apache.daffodil.runtime1.api.ComplexElementMetadata;
+import org.apache.daffodil.runtime1.api.ElementMetadata;
+import org.apache.daffodil.runtime1.api.InfosetArray;
+import org.apache.daffodil.runtime1.api.InfosetComplexElement;
+import org.apache.daffodil.japi.infoset.InfosetOutputter;
+import org.apache.daffodil.runtime1.api.InfosetSimpleElement;
+import org.apache.daffodil.runtime1.api.PrimitiveType;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.store.daffodil.schema.DrillDaffodilSchemaUtils;
+import org.apache.drill.exec.store.daffodil.schema.DrillDaffodilSchemaVisitor;
+import org.apache.drill.exec.vector.accessor.ArrayWriter;
+import org.apache.drill.exec.vector.accessor.ColumnWriter;
+import org.apache.drill.exec.vector.accessor.ObjectType;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Stack;
+
+/**
+ * Adapts Daffodil parser infoset event calls to Drill writer calls
+ * to fill in Drill data rows.
+ */
+public class DaffodilDrillInfosetOutputter
+    extends InfosetOutputter {
+
+  private boolean isOriginalRoot() {
+    boolean result = currentTupleWriter() == rowSetWriter;
+    if (result)
+      assert(tupleWriterStack.size() == 1);
+    return result;
+  }
+
+  /**
+   * True if the next startComplex call will be for the
+   * DFDL infoset root element whose children are the columns of
+   * the row set.
+   */
+  private boolean isRootElement = true;
+
+  /**
+   * Stack that is used only if we have sub-structures that are not
+   * simple-type fields of the row.
+   */
+  private final Stack<TupleWriter> tupleWriterStack = new Stack<>();
+
+  private final Stack<ArrayWriter> arrayWriterStack = new Stack<>();
+
+  private TupleWriter currentTupleWriter() {
+    return tupleWriterStack.peek();
+  }
+
+  private ArrayWriter currentArrayWriter() {
+    return arrayWriterStack.peek();
+  }
+
+
+  private static final Logger logger = LoggerFactory.getLogger(DaffodilDrillInfosetOutputter.class);
+
+  private DaffodilDrillInfosetOutputter() {} // no default constructor
+
+  private RowSetLoader rowSetWriter;
+
+  public DaffodilDrillInfosetOutputter(RowSetLoader writer) {
+    this.rowSetWriter = writer;
+    this.tupleWriterStack.push(writer);
+  }
+
+  @Override
+  public void reset() {
+    tupleWriterStack.clear();
+    tupleWriterStack.push(rowSetWriter);
+    arrayWriterStack.clear();
+    this.isRootElement = true;
+    checkCleanState();
+  }
+
+  private void checkCleanState() {
+    assert(isOriginalRoot());
+    assert(arrayWriterStack.isEmpty());
+    assert(isRootElement);
+  }
+
+  @Override
+  public void startDocument() {
+    checkCleanState();
+  }
+
+  @Override
+  public void endDocument() {
+    checkCleanState();
+  }
+
+  private String colName(ElementMetadata md) {
+    return DrillDaffodilSchemaVisitor.makeColumnName(md);
+  }
+
+  @Override
+  public void startSimple(InfosetSimpleElement ise) {
+    assert (!isRootElement);
+    ElementMetadata md = ise.metadata();
+    String colName = colName(md);
+    ColumnWriter cw;
+    if (md.isArray()) {
+      // A simple type array
+      assert(!arrayWriterStack.isEmpty());
+      cw = currentArrayWriter().scalar();
+    } else {
+      // A simple element within a map
+      // Note the map itself might be an array
+      // but we don't care about that here.
+      cw = currentTupleWriter().column(colName);
+    }
+    ColumnMetadata cm = cw.schema();
+    assert(cm.isScalar());
+    if (md.isNillable() && ise.isNilled()) {
+      assert cm.isNullable();
+      cw.setNull();
+    } else {
+      convertDaffodilValueToDrillValue(ise, cm, cw);
+    }
+  }
+
+  @Override
+  public void endSimple(InfosetSimpleElement diSimple) {
+    assert (!isRootElement);
+    // do nothing
+  }
+
+  @Override
+  public void startComplex(InfosetComplexElement ce) {
+    ComplexElementMetadata md = ce.metadata();
+    String colName = colName(ce.metadata());
+    if (isRootElement) {
+      assert(isOriginalRoot());
+      // This complex element's corresponds to the root element of the
+      // DFDL schema. We don't treat this as a column of the row set.
+      // Rather, it's children are the columns of the row set.
+      //
+      // If we do nothing at all here, then we'll start getting
+      // even calls for the children.
+      isRootElement = false;
+      return;
+    }
+    if (md.isArray()) {
+      assert(!arrayWriterStack.isEmpty());
+      // FIXME: is this the way to add a complex array child item (i.e., each array item is a map)
+      tupleWriterStack.push(currentArrayWriter().tuple());
+    } else {
+      tupleWriterStack.push(currentTupleWriter().tuple(colName));
+    }
+  }
+
+  @Override
+  public void endComplex(InfosetComplexElement ce) {
+    ComplexElementMetadata md = ce.metadata();
+    if (isOriginalRoot()) {
+      isRootElement = true;
+      // do nothing else. The row gets closed-out in the DaffodilBatchReader.next() method.
+    } else {
+      // it's a map.
+      // We seem to not need to do anything to end the map. No action taken here works.
+      if (md.isArray()) {
+        assert (!arrayWriterStack.isEmpty());
+        currentArrayWriter().save(); // required for map array entries.
+      }
+      tupleWriterStack.pop();
+    }
+  }
+
+  @Override
+  public void startArray(InfosetArray diArray) {
+    ElementMetadata md = diArray.metadata();
+    assert (md.isArray());
+    // DFDL has no notion of an array directly within another array. A named field (map) is necessary
+    // before you can have another array.
+    assert (currentTupleWriter().type() == ObjectType.TUPLE); // parent is a map, or the top level row.
+    String colName = colName(md);
+    TupleWriter enclosingParentTupleWriter = currentTupleWriter();
+    ArrayWriter aw = enclosingParentTupleWriter.array(colName);
+    arrayWriterStack.push(aw);
+  }
+
+  @Override
+  public void endArray(InfosetArray ia) {
+    ElementMetadata md = ia.metadata();
+    assert (md.isArray());
+    assert (!arrayWriterStack.empty());
+    // FIXME: How do we end/close-out an array?
+    // note that each array instance, when the instance is a map, must have
+    // save called after it is written to the array but that happens
+    // in endComplex events since it must be called not once per array, but
+    // once per array item.
+    arrayWriterStack.pop();
+  }
+
+  private void convertDaffodilValueToDrillValue(InfosetSimpleElement ise, ColumnMetadata cm, ColumnWriter cw) {
+    PrimitiveType dafType = ise.metadata().primitiveType();
+    TypeProtos.MinorType drillType = DrillDaffodilSchemaUtils.getDrillDataType(dafType);
+    assert(drillType == cm.type());
+    switch (drillType) {
+    case INT: {
+      //
+      // FIXME: Javadoc for setObject says "primarily for testing"
+      // So how are we supposed to assign the column value then?
+      // Is there a way to get from a ColumnWriter to a typed scalar writer (downcast perhaps?)
+      cw.setObject(ise.getInt());

Review Comment:
   @mbeckerle I did some experimenting here and have a suggestion.  
   
   Change the input to this function to accept a `ScalarWriter`.
   ```java
   private void convertDaffodilValueToDrillValue(InfosetSimpleElement ise, ColumnMetadata cm, ScalarWriter cw) { 
   ```
   
   Once you've done that, you can then access all the `setInt()` methods.  For additional fun, Drill has a few objects called `ScalarWriter`.   Be sure to import this one: `import org.apache.drill.exec.vector.accessor.ScalarWriter;`



-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] DRILL-8474: Add Daffodil Format Plugin (drill)

Posted by "mbeckerle (via GitHub)" <gi...@apache.org>.
mbeckerle commented on PR #2836:
URL: https://github.com/apache/drill/pull/2836#issuecomment-1878896878

   > @mbeckerle I had a thought about your TODO list. See inline.
   > 
   > > This is ready for a next review. All the scalar types are now implemented with typed setter calls.
   > > The prior review comments have all been addressed I believe.
   > > Remaining things to do include:
   > > 
   > > 1. How to get the compiled DFDL schema object so it can be loaded by daffodil out at the distributed Drill nodes.
   > 
   > I was thinking about this and I remembered something that might be useful. Drill has support for User Defined Functions (UDF) which are written in Java. To add a UDF to Drill, you also have to write some Java classes in a particular way, and include the JARs. Much like the DFDL class files, the UDF JARs must be accessible to all nodes of a Drill cluster.
   > 
   > Additionally, Drill has the capability of adding UDFs dynamically. This feature was added here: #574. Anyway, I wonder if we could use a similar mechanism to load and store the DFDL files so that they are accessible to all Drill nodes. What do you think?
   
   Excellent: So drill has all the machinery, it's just a question of repackaging it so it's available for this usage pattern, which is a bit different from Drill's UDFs, but also very similar. 
   
   There are two user scenarios which we can call production and test.
   
   1. Production: binary compiled DFDL schema file + code jars for Daffodil's own UDFs and "layers" plugins. This should, ideally, cache the compiled schema and not reload it for every query (at every node), but keep the same loaded instance in memory in a persistant JVM image on each node. For large production DFDL schemas this is the only sensible mechanism as it can take minutes to compile large DFDL schemas. 
   
   2. Test: on-the-fly centralized compilation of DFDL schema (from a combination of jars and files) to create and cache (to avoid recompiling) the binary compiled DFDL schema file. Then using that compiled binary file, as item 1. For small DFDL schemas this can be fast enough for production use. Ideally, if the DFDL schema is unchanged this would reuse the compiled binary file, but that's an optimization that may not matter much. 
   
   Kinds of objects involved are:
   
   - Daffodil plugin code jars
   - DFDL schema jars
   - DFDL schema files (just not packaged into a jar)
   - Daffodil compiled schema binary file
   - Daffodil config file - parameters, tunables, and options needed at compile time and/or runtime
   
   Code jars: Daffodil provides two extension features for DFDL users - DFDL UDFs and DFDL 'layers' (ex: plug-ins for uudecode, or gunzip algorithms used in part of the data format). Those are ordinary compiled class files in jars, so in all scenarios those jars are needed on the node class path if the DFDL schema uses them. Daffodil dynamically finds and loads these from the classpath in regular Java Service-Provider Interface (SPI) mechanisms. 
   
   Schema jars: Daffodil packages DFDL schema files (source files i.e., mySchema.dfdl.xsd) into jar files to allow inter-schema dependencies to be managed using ordinary jar/java-style managed dependencies. Tools like sbt and maven can express the dependencies of one schema on another, grab and pull them together, etc. Daffodil has a resolver so when one schema file referenes another with include/import it searches the class path directories and jars for the files. 
   
   Schema jars are only needed centrally when compiling the schema to a binary file. All references to the jar files for inter-schema file references are compiled into the compiled binary file. 
   
   It is possible for one DFDL schema 'project' to define a DFDL schema, along with the code for a plugin like a Daffodil UDF or layer. In that case the one jar created is both a code jar and a schema jar. The schema jar aspects are used when the schema is compiled and ignored at Daffodil runtime. The code jar aspects are used at Daffodil run time and ignored at schema compilation time. So such a jar that is both code and schema jar needs to be on the class path in both places, but there's no interaction of the two things. 
   
   Binary Compiled Schema File: Centrally, DFDL schemas in files and/or jars are compiled to create a single binary object which can be reloaded in order to actually use the schema to parse/unparse data. 
   
   - These binary files are tied to a specific version+build of Daffodil. (They are just a java object serialization of the runtime data structures used by Daffodil). 
   - Once reloaded into a JVM to create a Daffodil DataProcessor object, that object is read-only so thread safe, and can be shared by parse calls happening on many threads. 
   
   Daffodil Config File: This contains settings like what warnings to suppress when compiling and/or at runtime, tunables, such as how large to allow a regex match attempt, maximum parsed data size limit, etc. This also is needed both at schema compile and at runtime, as the same file contains parameters for both DFDL schema compile time and runtime.  
   
   
   
   


-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] DRILL-2835: Daffodil Feature for Drill (drill)

Posted by "cgivre (via GitHub)" <gi...@apache.org>.
cgivre commented on PR #2836:
URL: https://github.com/apache/drill/pull/2836#issuecomment-1867113844

   > Rebased onto latest Drill master as of 2023-12-21 (force pushed one more time)
   > 
   > Note that this is never going to pass automated tests until the Daffodil release this depends on is official (currently it needs a locally build Daffodil 3.7.0-snapshot, though the main daffodil branch has the changes integrated so any 3.7.0-snapshot build will work.
   
   @mbeckerle This is really great work!  Thanks for your persistence on this.  Do you have a an ETA on the next Daffodil release?


-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] DRILL-8474: Add Daffodil Format Plugin (drill)

Posted by "cgivre (via GitHub)" <gi...@apache.org>.
cgivre commented on code in PR #2836:
URL: https://github.com/apache/drill/pull/2836#discussion_r1451757410


##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/schema/DrillDaffodilSchemaVisitor.java:
##########
@@ -0,0 +1,229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.drill.exec.store.daffodil.schema;
+
+import org.apache.daffodil.runtime1.api.ChoiceMetadata;
+import org.apache.daffodil.runtime1.api.ComplexElementMetadata;
+import org.apache.daffodil.runtime1.api.ElementMetadata;
+import org.apache.daffodil.runtime1.api.InfosetSimpleElement;
+import org.apache.daffodil.runtime1.api.MetadataHandler;
+import org.apache.daffodil.runtime1.api.SequenceMetadata;
+import org.apache.daffodil.runtime1.api.SimpleElementMetadata;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.record.metadata.MapBuilder;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Stack;
+
+/**
+ * This class transforms a DFDL/Daffodil schema into a Drill Schema.
+ */
+public class DrillDaffodilSchemaVisitor extends MetadataHandler {
+  private static final Logger logger = LoggerFactory.getLogger(DrillDaffodilSchemaVisitor.class);
+  /**
+   * Unfortunately, SchemaBuilder and MapBuilder, while similar, do not share a base class so we
+   * have a stack of MapBuilders, and when empty we use the SchemaBuilder

Review Comment:
   This is likely music to @paul-rogers's ears.



-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] WIP: Preliminary Review on adding Daffodil to Drill (drill)

Posted by "cgivre (via GitHub)" <gi...@apache.org>.
cgivre commented on code in PR #2836:
URL: https://github.com/apache/drill/pull/2836#discussion_r1376780198


##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilBatchReader.java:
##########
@@ -64,64 +69,97 @@ public DaffodilBatchReader (DaffodilReaderConfig readerConfig, EasySubScan scan,
     this.validationMode = formatConfig.getValidationMode();
 
     //
-    // FIXME: Next, a MIRACLE occurs.
+    // FIXME: Where is this config file to be found? And, what is its syntax?

Review Comment:
   I'm feeling like we're crossing threads here a bit so let me back up a bit and explain how Drill handles configurations.  When I was talking about configs, I was talking about the params that the format plugin needs.  
   
   ## Format Configurations
   When you create a format plugin, the first file you likely created was `DaffodilFormatPlugin` which extends the `EasyFormatPlugin` interface.  Inside that generic, you added `DaffodilFormatConfig`. 
   
   ```java
   public class DaffodilFormatPlugin extends EasyFormatPlugin<DaffodilFormatConfig>
   ```
   
   By doing this, you've created the format plugin and associated it with a configuration object: `DaffodilFormatConfig`.   We do have the convention of calling these configs: `XXXFormatConfig` or `XXXStorageConfig`, but you could really call it whatever you want as long as that class implements the `FormatPluginConfig` interface.
   
   Let's say that we have a format called `foo`, and we've defined one variable called `bar` in the `FooFormatConfig` class.   Whenever you create a new instance of a file system connection (like HFDS, CP, dfs, etc.) that file system configuration has a list of formats which looks something like this:
   
   ```json
    "formats" : {
           "psv" : {
             "type" : "text",
             "extensions" : [ "tbl" ],
             "fieldDelimiter" : "|"
           },
           "csv" : {
             "type" : "text",
             "extensions" : [ "csv" ],
             "fieldDelimiter" : ",",
             "extractHeaders": true
           }
   }
   ```
   
   Using the Drill UI, the user can configure these parameters.  Drill will store the actual values as a json file in zookeeper, however, that process is handled by Drill's internals and isn't something that the format plugin has to manage.  A user should never manually edit these files directly.  They should only do so via Drill's UI, and there is a checksum to enforce that.
   
   As mentioned earlier, the user can override these parameters at query time by using the `table()` functions.  
   
   Drill will handle loading the configuration from this json file for you, and the daffodil format plugin does not need to do anything for that. 
   
   Bottom line is that any parameters you define in your config class, should be available via `plugin.getConfig()`.  Note that there is also a readerConfig object.  This is meant for more complex plugins and i don't think we need to use it for anything in this example.
   
   Does this help?  I hope this answers your questions.



-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] WIP: Preliminary Review on adding Daffodil to Drill (drill)

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


##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilBatchReader.java:
##########
@@ -64,64 +69,97 @@ public DaffodilBatchReader (DaffodilReaderConfig readerConfig, EasySubScan scan,
     this.validationMode = formatConfig.getValidationMode();
 
     //
-    // FIXME: Next, a MIRACLE occurs.
+    // FIXME: Where is this config file to be found? And, what is its syntax?

Review Comment:
   That "file" is scala code. You can't mean that's where users put the config info.
   
   Where exactly is the user-created config file found, what naming convention, and what syntax? properties? hocon? json?
   



-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] DRILL-2835: Daffodil Feature for Drill (drill)

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


##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilBatchReader.java:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.drill.exec.store.daffodil;
+
+import java.io.InputStream;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Objects;
+
+import org.apache.daffodil.japi.DataProcessor;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.physical.impl.scan.v3.ManagedReader;
+import org.apache.drill.exec.physical.impl.scan.v3.file.FileDescrip;
+import org.apache.drill.exec.physical.impl.scan.v3.file.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.store.daffodil.schema.DaffodilDataProcessorFactory;
+import org.apache.drill.exec.store.dfs.DrillFileSystem;
+import org.apache.drill.exec.store.dfs.easy.EasySubScan;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.drill.exec.store.daffodil.schema.DrillDaffodilSchemaUtils.daffodilDataProcessorToDrillSchema;
+
+
+public class DaffodilBatchReader implements ManagedReader {
+
+  private static final Logger logger = LoggerFactory.getLogger(DaffodilBatchReader.class);
+  private final DaffodilFormatConfig dafConfig;
+  private final RowSetLoader rowSetLoader;
+  private final CustomErrorContext errorContext;
+  private final DaffodilMessageParser dafParser;
+  private final InputStream dataInputStream;
+
+  static class DaffodilReaderConfig {
+    final DaffodilFormatPlugin plugin;
+    DaffodilReaderConfig(DaffodilFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public DaffodilBatchReader (DaffodilReaderConfig readerConfig, EasySubScan scan, FileSchemaNegotiator negotiator) {
+
+    errorContext = negotiator.parentErrorContext();
+    this.dafConfig = readerConfig.plugin.getConfig();
+
+    String schemaURIString = dafConfig.getSchemaURI(); // "schema/complexArray1.dfdl.xsd";
+    String rootName = dafConfig.getRootName();
+    String rootNamespace = dafConfig.getRootNamespace();
+    boolean validationMode = dafConfig.getValidationMode();
+
+    URI dfdlSchemaURI;
+    try {
+      dfdlSchemaURI = new URI(schemaURIString);
+    } catch (URISyntaxException e) {
+      throw UserException.validationError(e)
+          .build(logger);
+    }
+
+    FileDescrip file = negotiator.file();
+    DrillFileSystem fs = file.fileSystem();
+    URI fsSchemaURI = fs.getUri().resolve(dfdlSchemaURI);
+
+
+    DaffodilDataProcessorFactory dpf = new DaffodilDataProcessorFactory();
+    DataProcessor dp;
+    try {
+      dp = dpf.getDataProcessor(fsSchemaURI, validationMode, rootName, rootNamespace);
+    } catch (Exception e) {
+      throw UserException.dataReadError(e)
+          .message(String.format("Failed to get Daffodil DFDL processor for: %s", fsSchemaURI))
+          .addContext(errorContext).addContext(e.getMessage()).build(logger);
+    }
+    // Create the corresponding Drill schema.
+    // Note: this could be a very large schema. Think of a large complex RDBMS schema,
+    // all of it, hundreds of tables, but all part of the same metadata tree.
+    TupleMetadata drillSchema = daffodilDataProcessorToDrillSchema(dp);
+    // Inform Drill about the schema
+    negotiator.tableSchema(drillSchema, true);
+
+    //
+    // DATA TIME: Next we construct the runtime objects, and open files.
+    //
+    // We get the DaffodilMessageParser, which is a stateful driver for daffodil that
+    // actually does the parsing.
+    rowSetLoader = negotiator.build().writer();
+
+    // We construct the Daffodil InfosetOutputter which the daffodil parser uses to
+    // convert infoset event calls to fill in a Drill row via a rowSetLoader.
+    DaffodilDrillInfosetOutputter outputter = new DaffodilDrillInfosetOutputter(rowSetLoader);
+
+    // Now we can setup the dafParser with the outputter it will drive with
+    // the parser-produced infoset.
+    dafParser = new DaffodilMessageParser(dp); // needs further initialization after this.
+    dafParser.setInfosetOutputter(outputter);
+
+    Path dataPath = file.split().getPath();
+    // Lastly, we open the data stream
+    try {
+      dataInputStream = fs.openPossiblyCompressedStream(dataPath);
+    } catch (Exception e) {

Review Comment:
   Make narrower catch



##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilDrillInfosetOutputter.java:
##########
@@ -0,0 +1,296 @@
+/*
+ * 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.drill.exec.store.daffodil;
+
+import org.apache.daffodil.runtime1.api.ComplexElementMetadata;
+import org.apache.daffodil.runtime1.api.ElementMetadata;
+import org.apache.daffodil.runtime1.api.InfosetArray;
+import org.apache.daffodil.runtime1.api.InfosetComplexElement;
+import org.apache.daffodil.japi.infoset.InfosetOutputter;
+import org.apache.daffodil.runtime1.api.InfosetSimpleElement;
+import org.apache.daffodil.runtime1.api.PrimitiveType;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.store.daffodil.schema.DrillDaffodilSchemaUtils;
+import org.apache.drill.exec.store.daffodil.schema.DrillDaffodilSchemaVisitor;
+import org.apache.drill.exec.vector.accessor.ArrayWriter;
+import org.apache.drill.exec.vector.accessor.ColumnWriter;
+import org.apache.drill.exec.vector.accessor.ObjectType;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Stack;
+
+/**
+ * Adapts Daffodil parser infoset event calls to Drill writer calls
+ * to fill in Drill data rows.
+ */
+public class DaffodilDrillInfosetOutputter
+    extends InfosetOutputter {
+
+  private boolean isOriginalRoot() {
+    boolean result = currentTupleWriter() == rowSetWriter;
+    if (result)
+      assert(tupleWriterStack.size() == 1);
+    return result;
+  }
+
+  /**
+   * True if the next startComplex call will be for the
+   * DFDL infoset root element whose children are the columns of
+   * the row set.
+   */
+  private boolean isRootElement = true;
+
+  /**
+   * Stack that is used only if we have sub-structures that are not
+   * simple-type fields of the row.
+   */
+  private final Stack<TupleWriter> tupleWriterStack = new Stack<>();
+
+  private final Stack<ArrayWriter> arrayWriterStack = new Stack<>();
+
+  private TupleWriter currentTupleWriter() {
+    return tupleWriterStack.peek();
+  }
+
+  private ArrayWriter currentArrayWriter() {
+    return arrayWriterStack.peek();
+  }
+
+
+  private static final Logger logger = LoggerFactory.getLogger(DaffodilDrillInfosetOutputter.class);
+
+  private DaffodilDrillInfosetOutputter() {} // no default constructor
+
+  private RowSetLoader rowSetWriter;
+
+  public DaffodilDrillInfosetOutputter(RowSetLoader writer) {
+    this.rowSetWriter = writer;
+    this.tupleWriterStack.push(writer);
+  }
+
+  @Override
+  public void reset() {
+    tupleWriterStack.clear();
+    tupleWriterStack.push(rowSetWriter);
+    arrayWriterStack.clear();
+    this.isRootElement = true;
+    checkCleanState();
+  }
+
+  private void checkCleanState() {
+    assert(isOriginalRoot());
+    assert(arrayWriterStack.isEmpty());
+    assert(isRootElement);
+  }
+
+  @Override
+  public void startDocument() {
+    checkCleanState();
+  }
+
+  @Override
+  public void endDocument() {
+    checkCleanState();
+  }
+
+  private String colName(ElementMetadata md) {
+    return DrillDaffodilSchemaVisitor.makeColumnName(md);
+  }
+
+  @Override
+  public void startSimple(InfosetSimpleElement ise) {
+    assert (!isRootElement);
+    ElementMetadata md = ise.metadata();
+    String colName = colName(md);
+    ColumnWriter cw;
+    if (md.isArray()) {
+      // A simple type array
+      assert(!arrayWriterStack.isEmpty());
+      cw = currentArrayWriter().scalar();
+    } else {
+      // A simple element within a map
+      // Note the map itself might be an array
+      // but we don't care about that here.
+      cw = currentTupleWriter().column(colName);
+    }
+    ColumnMetadata cm = cw.schema();
+    assert(cm.isScalar());
+    if (md.isNillable() && ise.isNilled()) {
+      assert cm.isNullable();
+      cw.setNull();
+    } else {
+      convertDaffodilValueToDrillValue(ise, cm, cw);
+    }
+  }
+
+  @Override
+  public void endSimple(InfosetSimpleElement diSimple) {
+    assert (!isRootElement);
+    // do nothing
+  }
+
+  @Override
+  public void startComplex(InfosetComplexElement ce) {
+    ComplexElementMetadata md = ce.metadata();
+    String colName = colName(ce.metadata());
+    if (isRootElement) {
+      assert(isOriginalRoot());
+      // This complex element's corresponds to the root element of the
+      // DFDL schema. We don't treat this as a column of the row set.
+      // Rather, it's children are the columns of the row set.
+      //
+      // If we do nothing at all here, then we'll start getting
+      // even calls for the children.
+      isRootElement = false;
+      return;
+    }
+    if (md.isArray()) {
+      assert(!arrayWriterStack.isEmpty());
+      // FIXME: is this the way to add a complex array child item (i.e., each array item is a map)
+      tupleWriterStack.push(currentArrayWriter().tuple());
+    } else {
+      tupleWriterStack.push(currentTupleWriter().tuple(colName));
+    }
+  }
+
+  @Override
+  public void endComplex(InfosetComplexElement ce) {
+    ComplexElementMetadata md = ce.metadata();
+    if (isOriginalRoot()) {
+      isRootElement = true;
+      // do nothing else. The row gets closed-out in the DaffodilBatchReader.next() method.
+    } else {
+      // it's a map.
+      // We seem to not need to do anything to end the map. No action taken here works.
+      if (md.isArray()) {
+        assert (!arrayWriterStack.isEmpty());
+        currentArrayWriter().save(); // required for map array entries.
+      }
+      tupleWriterStack.pop();
+    }
+  }
+
+  @Override
+  public void startArray(InfosetArray diArray) {
+    ElementMetadata md = diArray.metadata();
+    assert (md.isArray());
+    // DFDL has no notion of an array directly within another array. A named field (map) is necessary
+    // before you can have another array.
+    assert (currentTupleWriter().type() == ObjectType.TUPLE); // parent is a map, or the top level row.
+    String colName = colName(md);
+    TupleWriter enclosingParentTupleWriter = currentTupleWriter();
+    ArrayWriter aw = enclosingParentTupleWriter.array(colName);
+    arrayWriterStack.push(aw);
+  }
+
+  @Override
+  public void endArray(InfosetArray ia) {
+    ElementMetadata md = ia.metadata();
+    assert (md.isArray());
+    assert (!arrayWriterStack.empty());
+    // FIXME: How do we end/close-out an array?
+    // note that each array instance, when the instance is a map, must have
+    // save called after it is written to the array but that happens
+    // in endComplex events since it must be called not once per array, but
+    // once per array item.
+    arrayWriterStack.pop();
+  }
+
+  private void convertDaffodilValueToDrillValue(InfosetSimpleElement ise, ColumnMetadata cm, ColumnWriter cw) {
+    PrimitiveType dafType = ise.metadata().primitiveType();
+    TypeProtos.MinorType drillType = DrillDaffodilSchemaUtils.getDrillDataType(dafType);
+    assert(drillType == cm.type());
+    switch (drillType) {
+    case INT: {
+      //
+      // FIXME: Javadoc for setObject says "primarily for testing"
+      // So how are we supposed to assign the column value then?
+      // Is there a way to get from a ColumnWriter to a typed scalar writer (downcast perhaps?)
+      cw.setObject(ise.getInt());

Review Comment:
   I did it this way because I could not find a way to get from these objects (cm, cw) to a ScalarWriter or something with a typed setter. I did look for that. 
   
   Can you look at this code and the call to it? Someplace I'm missing a step so that I have access to the objects with the typed set methods. 
    



##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilBatchReader.java:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.drill.exec.store.daffodil;
+
+import java.io.InputStream;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Objects;
+
+import org.apache.daffodil.japi.DataProcessor;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.physical.impl.scan.v3.ManagedReader;
+import org.apache.drill.exec.physical.impl.scan.v3.file.FileDescrip;
+import org.apache.drill.exec.physical.impl.scan.v3.file.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.store.daffodil.schema.DaffodilDataProcessorFactory;
+import org.apache.drill.exec.store.dfs.DrillFileSystem;
+import org.apache.drill.exec.store.dfs.easy.EasySubScan;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.drill.exec.store.daffodil.schema.DrillDaffodilSchemaUtils.daffodilDataProcessorToDrillSchema;
+
+
+public class DaffodilBatchReader implements ManagedReader {
+
+  private static final Logger logger = LoggerFactory.getLogger(DaffodilBatchReader.class);
+  private final DaffodilFormatConfig dafConfig;
+  private final RowSetLoader rowSetLoader;
+  private final CustomErrorContext errorContext;
+  private final DaffodilMessageParser dafParser;
+  private final InputStream dataInputStream;
+
+  static class DaffodilReaderConfig {
+    final DaffodilFormatPlugin plugin;
+    DaffodilReaderConfig(DaffodilFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public DaffodilBatchReader (DaffodilReaderConfig readerConfig, EasySubScan scan, FileSchemaNegotiator negotiator) {
+
+    errorContext = negotiator.parentErrorContext();
+    this.dafConfig = readerConfig.plugin.getConfig();
+
+    String schemaURIString = dafConfig.getSchemaURI(); // "schema/complexArray1.dfdl.xsd";
+    String rootName = dafConfig.getRootName();
+    String rootNamespace = dafConfig.getRootNamespace();
+    boolean validationMode = dafConfig.getValidationMode();
+
+    URI dfdlSchemaURI;
+    try {
+      dfdlSchemaURI = new URI(schemaURIString);
+    } catch (URISyntaxException e) {
+      throw UserException.validationError(e)
+          .build(logger);
+    }
+
+    FileDescrip file = negotiator.file();
+    DrillFileSystem fs = file.fileSystem();
+    URI fsSchemaURI = fs.getUri().resolve(dfdlSchemaURI);
+
+
+    DaffodilDataProcessorFactory dpf = new DaffodilDataProcessorFactory();
+    DataProcessor dp;
+    try {
+      dp = dpf.getDataProcessor(fsSchemaURI, validationMode, rootName, rootNamespace);
+    } catch (Exception e) {
+      throw UserException.dataReadError(e)
+          .message(String.format("Failed to get Daffodil DFDL processor for: %s", fsSchemaURI))
+          .addContext(errorContext).addContext(e.getMessage()).build(logger);
+    }
+    // Create the corresponding Drill schema.
+    // Note: this could be a very large schema. Think of a large complex RDBMS schema,
+    // all of it, hundreds of tables, but all part of the same metadata tree.
+    TupleMetadata drillSchema = daffodilDataProcessorToDrillSchema(dp);
+    // Inform Drill about the schema
+    negotiator.tableSchema(drillSchema, true);
+
+    //
+    // DATA TIME: Next we construct the runtime objects, and open files.
+    //
+    // We get the DaffodilMessageParser, which is a stateful driver for daffodil that
+    // actually does the parsing.
+    rowSetLoader = negotiator.build().writer();
+
+    // We construct the Daffodil InfosetOutputter which the daffodil parser uses to
+    // convert infoset event calls to fill in a Drill row via a rowSetLoader.
+    DaffodilDrillInfosetOutputter outputter = new DaffodilDrillInfosetOutputter(rowSetLoader);
+
+    // Now we can setup the dafParser with the outputter it will drive with
+    // the parser-produced infoset.
+    dafParser = new DaffodilMessageParser(dp); // needs further initialization after this.
+    dafParser.setInfosetOutputter(outputter);
+
+    Path dataPath = file.split().getPath();
+    // Lastly, we open the data stream
+    try {
+      dataInputStream = fs.openPossiblyCompressedStream(dataPath);
+    } catch (Exception e) {
+      throw UserException.dataReadError(e)
+          .message(String.format("Failed to open input file: %s", dataPath.toString()))
+          .addContext(errorContext).addContext(e.getMessage()).build(logger);
+    }
+    // And lastly,... tell daffodil the input data stream.
+    dafParser.setInputStream(dataInputStream);
+  }
+
+
+  /**
+   * This is the core of actual processing - data movement from Daffodil to Drill.
+   * <p>
+   * If there is space in the batch, and there is data available to parse
+   * then this calls the daffodil parser, which parses data, delivering it to the rowWriter
+   * by way of the infoset outputter.
+   * <p>
+   * Repeats until the rowWriter is full (a batch is full), or there is no more data, or
+   * a parse error ends execution with a throw.
+   * <p>
+   * Validation errors and other warnings are not errors and are logged but do not cause
+   * parsing to fail/throw.
+   * @return true if there are rows retrieved, false if no rows were retrieved, which means
+   * no more will ever be retrieved (end of data).
+   * @throws RuntimeException on parse errors.
+   */
+  @Override
+  public boolean next() {
+    // Check assumed invariants
+    // We don't know if there is data or not. This could be called on an empty data file.
+    // We DO know that this won't be called if there is no space in the batch for even 1
+    // row.
+    if (dafParser.isEOF()) {
+      return false; // return without even checking for more rows or trying to parse.
+    }
+    while (rowSetLoader.start() && !dafParser.isEOF()) { // we never zero-trip this loop.
+      // the predicate is always true once.
+      try {
+        dafParser.parse();
+        if (dafParser.isProcessingError()) {
+          assert(Objects.nonNull(dafParser.getDiagnostics()));
+          throw UserException.dataReadError().message(dafParser.getDiagnosticsAsString())
+              .addContext(errorContext).build(logger);
+        }
+        if (dafParser.isValidationError()) {
+          logger.warn(dafParser.getDiagnosticsAsString());
+          // Note that even if daffodil is set to not validate, validation errors may still occur
+          // from DFDL's "recoverableError" assertions.
+        }
+      } catch (Exception e) {
+        throw UserException.dataReadError(e).message("Error parsing file: " + e.getMessage())
+            .addContext(errorContext).build(logger);
+      }
+      rowSetLoader.save();
+    }
+    int nRows = rowSetLoader.rowCount();

Review Comment:
   We don't need it, but I like to do this.  This is really only to make the API invariant clear, which is that Drill depends on the nRows not being zero (or rather, is allowed to depend on that). This is my understanding of how the Drill API works.  I inferred this from other code that I was using as a model when writing this code. 
   
   I like to test and make explicit as many such assumptions as possible, and I prefer code/assert to just comments to that effect. I also like to leave such tests in the code. It's only in inner loops or where profiling shows it matters that I would normally convert this into comments.  Fields are the 'inner loop' of this code area, not rows, I would claim. 



##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilBatchReader.java:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.drill.exec.store.daffodil;
+
+import java.io.InputStream;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Objects;
+
+import org.apache.daffodil.japi.DataProcessor;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.physical.impl.scan.v3.ManagedReader;
+import org.apache.drill.exec.physical.impl.scan.v3.file.FileDescrip;
+import org.apache.drill.exec.physical.impl.scan.v3.file.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.store.daffodil.schema.DaffodilDataProcessorFactory;
+import org.apache.drill.exec.store.dfs.DrillFileSystem;
+import org.apache.drill.exec.store.dfs.easy.EasySubScan;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.drill.exec.store.daffodil.schema.DrillDaffodilSchemaUtils.daffodilDataProcessorToDrillSchema;
+
+
+public class DaffodilBatchReader implements ManagedReader {
+
+  private static final Logger logger = LoggerFactory.getLogger(DaffodilBatchReader.class);
+  private final DaffodilFormatConfig dafConfig;
+  private final RowSetLoader rowSetLoader;
+  private final CustomErrorContext errorContext;
+  private final DaffodilMessageParser dafParser;
+  private final InputStream dataInputStream;
+
+  static class DaffodilReaderConfig {
+    final DaffodilFormatPlugin plugin;
+    DaffodilReaderConfig(DaffodilFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public DaffodilBatchReader (DaffodilReaderConfig readerConfig, EasySubScan scan, FileSchemaNegotiator negotiator) {
+
+    errorContext = negotiator.parentErrorContext();
+    this.dafConfig = readerConfig.plugin.getConfig();
+
+    String schemaURIString = dafConfig.getSchemaURI(); // "schema/complexArray1.dfdl.xsd";
+    String rootName = dafConfig.getRootName();
+    String rootNamespace = dafConfig.getRootNamespace();
+    boolean validationMode = dafConfig.getValidationMode();
+
+    URI dfdlSchemaURI;
+    try {
+      dfdlSchemaURI = new URI(schemaURIString);
+    } catch (URISyntaxException e) {
+      throw UserException.validationError(e)
+          .build(logger);
+    }
+
+    FileDescrip file = negotiator.file();
+    DrillFileSystem fs = file.fileSystem();
+    URI fsSchemaURI = fs.getUri().resolve(dfdlSchemaURI);
+
+
+    DaffodilDataProcessorFactory dpf = new DaffodilDataProcessorFactory();
+    DataProcessor dp;
+    try {
+      dp = dpf.getDataProcessor(fsSchemaURI, validationMode, rootName, rootNamespace);
+    } catch (Exception e) {
+      throw UserException.dataReadError(e)
+          .message(String.format("Failed to get Daffodil DFDL processor for: %s", fsSchemaURI))
+          .addContext(errorContext).addContext(e.getMessage()).build(logger);
+    }
+    // Create the corresponding Drill schema.
+    // Note: this could be a very large schema. Think of a large complex RDBMS schema,
+    // all of it, hundreds of tables, but all part of the same metadata tree.
+    TupleMetadata drillSchema = daffodilDataProcessorToDrillSchema(dp);
+    // Inform Drill about the schema
+    negotiator.tableSchema(drillSchema, true);
+
+    //
+    // DATA TIME: Next we construct the runtime objects, and open files.
+    //
+    // We get the DaffodilMessageParser, which is a stateful driver for daffodil that
+    // actually does the parsing.
+    rowSetLoader = negotiator.build().writer();
+
+    // We construct the Daffodil InfosetOutputter which the daffodil parser uses to
+    // convert infoset event calls to fill in a Drill row via a rowSetLoader.
+    DaffodilDrillInfosetOutputter outputter = new DaffodilDrillInfosetOutputter(rowSetLoader);
+
+    // Now we can setup the dafParser with the outputter it will drive with
+    // the parser-produced infoset.
+    dafParser = new DaffodilMessageParser(dp); // needs further initialization after this.
+    dafParser.setInfosetOutputter(outputter);
+
+    Path dataPath = file.split().getPath();
+    // Lastly, we open the data stream
+    try {
+      dataInputStream = fs.openPossiblyCompressedStream(dataPath);
+    } catch (Exception e) {
+      throw UserException.dataReadError(e)
+          .message(String.format("Failed to open input file: %s", dataPath.toString()))
+          .addContext(errorContext).addContext(e.getMessage()).build(logger);
+    }
+    // And lastly,... tell daffodil the input data stream.
+    dafParser.setInputStream(dataInputStream);
+  }
+
+
+  /**
+   * This is the core of actual processing - data movement from Daffodil to Drill.
+   * <p>
+   * If there is space in the batch, and there is data available to parse
+   * then this calls the daffodil parser, which parses data, delivering it to the rowWriter
+   * by way of the infoset outputter.
+   * <p>
+   * Repeats until the rowWriter is full (a batch is full), or there is no more data, or
+   * a parse error ends execution with a throw.
+   * <p>
+   * Validation errors and other warnings are not errors and are logged but do not cause
+   * parsing to fail/throw.
+   * @return true if there are rows retrieved, false if no rows were retrieved, which means
+   * no more will ever be retrieved (end of data).
+   * @throws RuntimeException on parse errors.
+   */
+  @Override
+  public boolean next() {
+    // Check assumed invariants
+    // We don't know if there is data or not. This could be called on an empty data file.
+    // We DO know that this won't be called if there is no space in the batch for even 1
+    // row.
+    if (dafParser.isEOF()) {
+      return false; // return without even checking for more rows or trying to parse.
+    }
+    while (rowSetLoader.start() && !dafParser.isEOF()) { // we never zero-trip this loop.
+      // the predicate is always true once.
+      try {
+        dafParser.parse();
+        if (dafParser.isProcessingError()) {
+          assert(Objects.nonNull(dafParser.getDiagnostics()));
+          throw UserException.dataReadError().message(dafParser.getDiagnosticsAsString())
+              .addContext(errorContext).build(logger);
+        }
+        if (dafParser.isValidationError()) {
+          logger.warn(dafParser.getDiagnosticsAsString());
+          // Note that even if daffodil is set to not validate, validation errors may still occur
+          // from DFDL's "recoverableError" assertions.
+        }
+      } catch (Exception e) {
+        throw UserException.dataReadError(e).message("Error parsing file: " + e.getMessage())
+            .addContext(errorContext).build(logger);
+      }
+      rowSetLoader.save();
+    }
+    int nRows = rowSetLoader.rowCount();
+    assert nRows > 0; // This cannot be zero. If the parse failed we will have already thrown out of here.
+    return true;
+  }
+
+  @Override
+  public void close() {
+    AutoCloseables.closeSilently(dataInputStream);

Review Comment:
   This is the only close required by Daffodil. 



##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilDrillInfosetOutputter.java:
##########
@@ -0,0 +1,296 @@
+/*
+ * 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.drill.exec.store.daffodil;
+
+import org.apache.daffodil.runtime1.api.ComplexElementMetadata;
+import org.apache.daffodil.runtime1.api.ElementMetadata;
+import org.apache.daffodil.runtime1.api.InfosetArray;
+import org.apache.daffodil.runtime1.api.InfosetComplexElement;
+import org.apache.daffodil.japi.infoset.InfosetOutputter;
+import org.apache.daffodil.runtime1.api.InfosetSimpleElement;
+import org.apache.daffodil.runtime1.api.PrimitiveType;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.store.daffodil.schema.DrillDaffodilSchemaUtils;
+import org.apache.drill.exec.store.daffodil.schema.DrillDaffodilSchemaVisitor;
+import org.apache.drill.exec.vector.accessor.ArrayWriter;
+import org.apache.drill.exec.vector.accessor.ColumnWriter;
+import org.apache.drill.exec.vector.accessor.ObjectType;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Stack;
+
+/**
+ * Adapts Daffodil parser infoset event calls to Drill writer calls
+ * to fill in Drill data rows.
+ */
+public class DaffodilDrillInfosetOutputter
+    extends InfosetOutputter {
+
+  private boolean isOriginalRoot() {
+    boolean result = currentTupleWriter() == rowSetWriter;
+    if (result)
+      assert(tupleWriterStack.size() == 1);
+    return result;
+  }
+
+  /**
+   * True if the next startComplex call will be for the
+   * DFDL infoset root element whose children are the columns of
+   * the row set.
+   */
+  private boolean isRootElement = true;
+
+  /**
+   * Stack that is used only if we have sub-structures that are not
+   * simple-type fields of the row.
+   */
+  private final Stack<TupleWriter> tupleWriterStack = new Stack<>();
+
+  private final Stack<ArrayWriter> arrayWriterStack = new Stack<>();
+
+  private TupleWriter currentTupleWriter() {
+    return tupleWriterStack.peek();
+  }
+
+  private ArrayWriter currentArrayWriter() {
+    return arrayWriterStack.peek();
+  }
+
+
+  private static final Logger logger = LoggerFactory.getLogger(DaffodilDrillInfosetOutputter.class);
+
+  private DaffodilDrillInfosetOutputter() {} // no default constructor
+
+  private RowSetLoader rowSetWriter;
+
+  public DaffodilDrillInfosetOutputter(RowSetLoader writer) {
+    this.rowSetWriter = writer;
+    this.tupleWriterStack.push(writer);
+  }
+
+  @Override
+  public void reset() {
+    tupleWriterStack.clear();
+    tupleWriterStack.push(rowSetWriter);
+    arrayWriterStack.clear();
+    this.isRootElement = true;
+    checkCleanState();
+  }
+
+  private void checkCleanState() {
+    assert(isOriginalRoot());
+    assert(arrayWriterStack.isEmpty());
+    assert(isRootElement);
+  }
+
+  @Override
+  public void startDocument() {
+    checkCleanState();
+  }
+
+  @Override
+  public void endDocument() {
+    checkCleanState();
+  }
+
+  private String colName(ElementMetadata md) {
+    return DrillDaffodilSchemaVisitor.makeColumnName(md);
+  }
+
+  @Override
+  public void startSimple(InfosetSimpleElement ise) {
+    assert (!isRootElement);
+    ElementMetadata md = ise.metadata();
+    String colName = colName(md);
+    ColumnWriter cw;
+    if (md.isArray()) {
+      // A simple type array
+      assert(!arrayWriterStack.isEmpty());
+      cw = currentArrayWriter().scalar();
+    } else {
+      // A simple element within a map
+      // Note the map itself might be an array
+      // but we don't care about that here.
+      cw = currentTupleWriter().column(colName);
+    }
+    ColumnMetadata cm = cw.schema();
+    assert(cm.isScalar());
+    if (md.isNillable() && ise.isNilled()) {
+      assert cm.isNullable();
+      cw.setNull();
+    } else {
+      convertDaffodilValueToDrillValue(ise, cm, cw);
+    }
+  }
+
+  @Override
+  public void endSimple(InfosetSimpleElement diSimple) {
+    assert (!isRootElement);
+    // do nothing
+  }
+
+  @Override
+  public void startComplex(InfosetComplexElement ce) {
+    ComplexElementMetadata md = ce.metadata();
+    String colName = colName(ce.metadata());
+    if (isRootElement) {
+      assert(isOriginalRoot());
+      // This complex element's corresponds to the root element of the
+      // DFDL schema. We don't treat this as a column of the row set.
+      // Rather, it's children are the columns of the row set.
+      //
+      // If we do nothing at all here, then we'll start getting
+      // even calls for the children.
+      isRootElement = false;
+      return;
+    }
+    if (md.isArray()) {
+      assert(!arrayWriterStack.isEmpty());
+      // FIXME: is this the way to add a complex array child item (i.e., each array item is a map)
+      tupleWriterStack.push(currentArrayWriter().tuple());
+    } else {
+      tupleWriterStack.push(currentTupleWriter().tuple(colName));
+    }
+  }
+
+  @Override
+  public void endComplex(InfosetComplexElement ce) {
+    ComplexElementMetadata md = ce.metadata();
+    if (isOriginalRoot()) {
+      isRootElement = true;
+      // do nothing else. The row gets closed-out in the DaffodilBatchReader.next() method.
+    } else {
+      // it's a map.
+      // We seem to not need to do anything to end the map. No action taken here works.
+      if (md.isArray()) {
+        assert (!arrayWriterStack.isEmpty());
+        currentArrayWriter().save(); // required for map array entries.
+      }
+      tupleWriterStack.pop();
+    }
+  }
+
+  @Override
+  public void startArray(InfosetArray diArray) {
+    ElementMetadata md = diArray.metadata();
+    assert (md.isArray());
+    // DFDL has no notion of an array directly within another array. A named field (map) is necessary
+    // before you can have another array.
+    assert (currentTupleWriter().type() == ObjectType.TUPLE); // parent is a map, or the top level row.
+    String colName = colName(md);
+    TupleWriter enclosingParentTupleWriter = currentTupleWriter();
+    ArrayWriter aw = enclosingParentTupleWriter.array(colName);
+    arrayWriterStack.push(aw);
+  }
+
+  @Override
+  public void endArray(InfosetArray ia) {
+    ElementMetadata md = ia.metadata();
+    assert (md.isArray());
+    assert (!arrayWriterStack.empty());
+    // FIXME: How do we end/close-out an array?
+    // note that each array instance, when the instance is a map, must have
+    // save called after it is written to the array but that happens
+    // in endComplex events since it must be called not once per array, but
+    // once per array item.
+    arrayWriterStack.pop();
+  }
+
+  private void convertDaffodilValueToDrillValue(InfosetSimpleElement ise, ColumnMetadata cm, ColumnWriter cw) {
+    PrimitiveType dafType = ise.metadata().primitiveType();
+    TypeProtos.MinorType drillType = DrillDaffodilSchemaUtils.getDrillDataType(dafType);
+    assert(drillType == cm.type());
+    switch (drillType) {
+    case INT: {
+      //
+      // FIXME: Javadoc for setObject says "primarily for testing"
+      // So how are we supposed to assign the column value then?
+      // Is there a way to get from a ColumnWriter to a typed scalar writer (downcast perhaps?)
+      cw.setObject(ise.getInt());
+      break;
+    }
+    case BIGINT: {
+      cw.setObject(ise.getLong());
+      break;
+    }
+    case SMALLINT: {
+      cw.setObject(ise.getShort());
+      break;
+    }
+    case TINYINT: {
+      cw.setObject(ise.getByte());
+      break;
+    }
+//        .put("UNSIGNEDLONG", TypeProtos.MinorType.UINT8)
+//        .put("UNSIGNEDINT", TypeProtos.MinorType.UINT4)
+//        .put("UNSIGNEDSHORT", TypeProtos.MinorType.UINT2)
+//        .put("UNSIGNEDBYTE", TypeProtos.MinorType.UINT1)
+//        .put("INTEGER", TypeProtos.MinorType.BIGINT)
+//        .put("NONNEGATIVEINTEGER", TypeProtos.MinorType.BIGINT)
+    case BIT: {
+      cw.setObject(ise.getBoolean());
+      break;
+    }
+//        .put("DATE", TypeProtos.MinorType.DATE) // requires conversion
+//        .put("DATETIME", TypeProtos.MinorType.TIMESTAMP) // requires conversion
+//        .put("DECIMAL", TypeProtos.MinorType.VARDECIMAL) // requires conversion (maybe)
+    case FLOAT8: {
+      cw.setObject(ise.getDouble());
+      break;
+    }
+    case FLOAT4: {
+      cw.setObject(ise.getFloat());
+      break;
+    }
+    case VARBINARY: {
+      cw.setObject(ise.getHexBinary());
+      break;
+    }
+    case VARCHAR: {
+      //
+      // FIXME: VARCHAR is defined in drill as utf8 string.
+      // Is Drill expecting something other than a Java string in this setObject call?
+      // Should we be mapping Daffodil strings to Drill VAR16CHAR type?
+      //
+      String s = ise.getString();
+      cw.setObject(s);
+      break;
+    }
+//        .put("TIME", TypeProtos.MinorType.TIME) // requires conversion

Review Comment:
   Yes Daffodil has date, time, and dateTime types.  At the API level we have essentially Java's ordinary calendar-related classes in hand at this point, so it should be straightforward to consume those when creating Drill's equivalent types. 
   
   However, I stubbed all this out until resolving the typed setter issue.



##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilDrillInfosetOutputter.java:
##########
@@ -0,0 +1,296 @@
+/*
+ * 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.drill.exec.store.daffodil;
+
+import org.apache.daffodil.runtime1.api.ComplexElementMetadata;
+import org.apache.daffodil.runtime1.api.ElementMetadata;
+import org.apache.daffodil.runtime1.api.InfosetArray;
+import org.apache.daffodil.runtime1.api.InfosetComplexElement;
+import org.apache.daffodil.japi.infoset.InfosetOutputter;
+import org.apache.daffodil.runtime1.api.InfosetSimpleElement;
+import org.apache.daffodil.runtime1.api.PrimitiveType;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.store.daffodil.schema.DrillDaffodilSchemaUtils;
+import org.apache.drill.exec.store.daffodil.schema.DrillDaffodilSchemaVisitor;
+import org.apache.drill.exec.vector.accessor.ArrayWriter;
+import org.apache.drill.exec.vector.accessor.ColumnWriter;
+import org.apache.drill.exec.vector.accessor.ObjectType;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Stack;
+
+/**
+ * Adapts Daffodil parser infoset event calls to Drill writer calls
+ * to fill in Drill data rows.
+ */
+public class DaffodilDrillInfosetOutputter
+    extends InfosetOutputter {
+
+  private boolean isOriginalRoot() {
+    boolean result = currentTupleWriter() == rowSetWriter;

Review Comment:
   Is the Drill coding style defined in a wiki or other doc page somewhere? I didn't find one. 
   
   If this is just java-standard, then I need reminding, as I have not coded Java prior to this effort for 12+ years now. 



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

To unsubscribe, e-mail: dev-unsubscribe@drill.apache.org

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


Re: [PR] DRILL-8474: Add Daffodil Format Plugin (drill)

Posted by "mbeckerle (via GitHub)" <gi...@apache.org>.
mbeckerle commented on PR #2836:
URL: https://github.com/apache/drill/pull/2836#issuecomment-1906561549

   > > @cgivre @paul-rogers is there an example of a Drill UDF that is not part of the drill repository tree?
   > > I'd like to understand the mechanisms for distributing any jar files and dependencies of the UDF that drill uses. I can't find any such in the quasi-USFs that are in the Drill tree, because well, since they are part of Drill, and so are their dependencies, this problem doesn't exist.
   > 
   > @mbeckerle Here's an example: https://github.com/datadistillr/drill-humanname-functions. I'm sorry we weren't able to connect last week.
   
   If I understand this correctly, if a jar is on the classpath and has drill-module.conf in its root dir, then drill will find it and read that HOCON file to get the package to add to drill.classpath.scanning.packages. 
   
   Drill then appears to scan jars for class files for those packages. Not sure what it is doing with the class files. I imagine it is repackaging them somehow so Drill can use them on the drill distributed nodes. But it isn't yet clear to me how this aspect works. Do these classes just get loaded on the distributed drill nodes? Or is the classpath augmented in some way on the drill nodes so that they see a jar that contains all these classes?
   
   I have two questions: 
   
   (1) what about dependencies? The UDF may depend on libraries which depend on other libraries, etc. 
   
   (2) what about non-class files, e.g., things under src/main/resources of the project that go into the jar, but aren't "class" files? How do those things also get moved? How would code running in the drill node access these? The usual method is to call getResource(URL) with a URL that gives the path within a jar file to the resource in question. 
   
   Thanks for any info. 
   


-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] DRILL-8474: Add Daffodil Format Plugin (drill)

Posted by "cgivre (via GitHub)" <gi...@apache.org>.
cgivre commented on PR #2836:
URL: https://github.com/apache/drill/pull/2836#issuecomment-1877962587

   @mbeckerle I had a thought about your TODO list.   See inline.
   
   > This is ready for a next review. All the scalar types are now implemented with typed setter calls.
   > 
   > The prior review comments have all been addressed I believe.
   > 
   > Remaining things to do include:
   > 
   > 1. How to get the compiled DFDL schema object so it can be loaded by daffodil out at the distributed Drill nodes.
   
   I was thinking about this and I remembered something that might be useful.  Drill has support for User Defined Functions (UDF) which are written in Java.  To add a UDF to Drill, you also have to write some Java classes in a particular way, and include the JARs.   Much like the DFDL class files, the UDF JARs must be accessible to all nodes of a Drill cluster. 
   
   Additionally, Drill has the capability of adding UDFs dynamically.   This feature was added here: https://github.com/apache/drill/pull/574.  Anyway, I wonder if we could use a similar mechanism to load and store the DFDL files so that they are accessible to all Drill nodes.  What do you think?
   
   
   > 2. Test of nilled values (and more tests generally to show deeply nested and repeating nested objects work.)
   > 3. Errors - revisit every place errors are detected or thrown to make sure these are being done the right way for DFDL schema compilation and runtime errors as well.
   
   


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

To unsubscribe, e-mail: dev-unsubscribe@drill.apache.org

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


Re: [PR] DRILL-8474: Add Daffodil Format Plugin (drill)

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


##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/schema/DrillDaffodilSchemaVisitor.java:
##########
@@ -0,0 +1,229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.drill.exec.store.daffodil.schema;
+
+import org.apache.daffodil.runtime1.api.ChoiceMetadata;
+import org.apache.daffodil.runtime1.api.ComplexElementMetadata;
+import org.apache.daffodil.runtime1.api.ElementMetadata;
+import org.apache.daffodil.runtime1.api.InfosetSimpleElement;
+import org.apache.daffodil.runtime1.api.MetadataHandler;
+import org.apache.daffodil.runtime1.api.SequenceMetadata;
+import org.apache.daffodil.runtime1.api.SimpleElementMetadata;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.record.metadata.MapBuilder;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Stack;
+
+/**
+ * This class transforms a DFDL/Daffodil schema into a Drill Schema.
+ */
+public class DrillDaffodilSchemaVisitor extends MetadataHandler {
+  private static final Logger logger = LoggerFactory.getLogger(DrillDaffodilSchemaVisitor.class);
+  /**
+   * Unfortunately, SchemaBuilder and MapBuilder, while similar, do not share a base class so we
+   * have a stack of MapBuilders, and when empty we use the SchemaBuilder

Review Comment:
   Note that this awkwardness effectively doubles the code size of things that interface to Drill. 
   
   This duplication of similar behavior for schema and map builders (and rowWriters and mapWriters) is expected and typical of systems that start from a tabular view of the data world and later add the features needed for hierachical data. Nevertheless it is awkward when one is dealing entirely with hierarchical data. 
   
   A MetaBuilder that does the map thing if the builder is a map, and the schema thing if the builder is a schema would eliminate this. This could be an interface mixed into both SchemaBuilder and MapBuilder (could also be called MapBuilderLike). 
   
   The same discontinuity at the base holds for RowWriter vs. MapWriter in the runtime handling of data. Again it doubles the code size/complexity, every fix goes in 2 places, etc. A MapWriterLike interface could be factored out.
   
   Maybe we should build such mechanisms to avoid this, and then use them to improve this Daffodil plugin?
   



##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/schema/DrillDaffodilSchemaUtils.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.drill.exec.store.daffodil.schema;
+
+import org.apache.daffodil.japi.InvalidParserException;
+import org.apache.daffodil.japi.DataProcessor;
+import org.apache.daffodil.runtime1.api.PrimitiveType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableMap;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+
+public class DrillDaffodilSchemaUtils {
+  private static final MinorType DEFAULT_TYPE = MinorType.VARCHAR;
+  private static final Logger logger = LoggerFactory.getLogger(DrillDaffodilSchemaUtils.class);
+
+  /**
+   * This map maps the data types defined by the DFDL definition to Drill data types.
+   */
+  public static final ImmutableMap<String, MinorType> DFDL_TYPE_MAPPINGS =
+      ImmutableMap.<String, MinorType>builder()
+          .put("LONG", MinorType.BIGINT)
+          .put("INT", MinorType.INT)
+          .put("SHORT", MinorType.SMALLINT)
+          .put("BYTE", MinorType.TINYINT)
+          // daffodil unsigned longs are modeled as DECIMAL(38, 0) which is the default for VARDECIMAL
+          .put("UNSIGNEDLONG", MinorType.VARDECIMAL)
+          .put("UNSIGNEDINT", MinorType.BIGINT)
+          .put("UNSIGNEDSHORT", MinorType.UINT2)
+          .put("UNSIGNEDBYTE", MinorType.UINT1)
+          // daffodil integer, nonNegativeInteger, are modeled as DECIMAL(38, 0) which is the default for VARDECIMAL
+          .put("INTEGER", MinorType.VARDECIMAL)
+          .put("NONNEGATIVEINTEGER", MinorType.VARDECIMAL)
+          // decimal has to be modeled as string since we really have no idea what to set the
+          // scale to.
+          .put("DECIMAL", MinorType.VARCHAR)
+          .put("BOOLEAN", MinorType.BIT)
+          .put("DATE", MinorType.DATE) // requires conversion
+          .put("DATETIME", MinorType.TIMESTAMP) // requires conversion
+          .put("DOUBLE", MinorType.FLOAT8)
+          //
+          // daffodil float type is mapped to double aka Float8 in drill because there
+          // seems to be bugs in FLOAT4. Float.MaxValue in a Float4 column displays as
+          // 3.4028234663852886E38 not 3.4028235E38.
+          //
+          // We don't really care about single float precision, so we just use double precision.
+          //
+          .put("FLOAT", MinorType.FLOAT8)
+          .put("HEXBINARY", MinorType.VARBINARY)
+          .put("STRING", MinorType.VARCHAR)
+          .put("TIME", MinorType.TIME) // requires conversion
+          .build();
+
+
+  @VisibleForTesting
+  public static TupleMetadata processSchema(URI dfdlSchemaURI, String rootName, String namespace)
+      throws IOException, DaffodilDataProcessorFactory.CompileFailure,
+      URISyntaxException, InvalidParserException {
+    DaffodilDataProcessorFactory dpf = new DaffodilDataProcessorFactory();
+    DataProcessor dp = dpf.getDataProcessor(dfdlSchemaURI, true, rootName, namespace);

Review Comment:
   Add 
   ```
   boolean validationMode = true // use Daffodil's limited validation always
   ```
   instead of just passing true. 
   
   Document in the README.md that limited validation is always performed. 
   TBD: Document how validation errors are handled - escalated to errors, or issued as warnings? Is an option needed here?



##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/schema/DaffodilDataProcessorFactory.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.drill.exec.store.daffodil.schema;
+
+import org.apache.daffodil.japi.Compiler;
+import org.apache.daffodil.japi.Daffodil;
+import org.apache.daffodil.japi.DataProcessor;
+import org.apache.daffodil.japi.Diagnostic;
+import org.apache.daffodil.japi.InvalidParserException;
+import org.apache.daffodil.japi.InvalidUsageException;
+import org.apache.daffodil.japi.ProcessorFactory;
+import org.apache.daffodil.japi.ValidationMode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.nio.channels.Channels;
+import java.util.List;
+import java.util.Objects;
+
+/**
+ * Compiles a DFDL schema (mostly for tests) or loads a pre-compiled DFDL schema so that one can
+ * obtain a DataProcessor for use with DaffodilMessageParser.
+ * <p/>
+ * TODO: Needs to use a cache to avoid reloading/recompiling every time.
+ */
+public class DaffodilDataProcessorFactory {
+  // Default constructor is used.
+
+  private static final Logger logger = LoggerFactory.getLogger(DaffodilDataProcessorFactory.class);
+
+  private DataProcessor dp;
+
+  /**
+   * Gets a Daffodil DataProcessor given the necessary arguments to compile or reload it.
+   *
+   * @param schemaFileURI
+   *     pre-compiled dfdl schema (.bin extension) or DFDL schema source (.xsd extension)
+   * @param validationMode
+   *     Use true to request Daffodil built-in 'limited' validation. Use false for no validation.
+   * @param rootName
+   *     Local name of root element of the message. Can be null to use the first element declaration
+   *     of the primary schema file. Ignored if reloading a pre-compiled schema.
+   * @param rootNS
+   *     Namespace URI as a string. Can be null to use the target namespace of the primary schema
+   *     file or if it is unambiguous what element is the rootName. Ignored if reloading a
+   *     pre-compiled schema.
+   * @return the DataProcessor
+   * @throws CompileFailure
+   *     - if schema compilation fails
+   */
+  public DataProcessor getDataProcessor(URI schemaFileURI, boolean validationMode, String rootName,
+      String rootNS)
+      throws CompileFailure {
+
+    DaffodilDataProcessorFactory dmp = new DaffodilDataProcessorFactory();
+    boolean isPrecompiled = schemaFileURI.toString().endsWith(".bin");
+    if (isPrecompiled) {
+      if (Objects.nonNull(rootName) && !rootName.isEmpty()) {
+        // A usage error. You shouldn't supply the name and optionally namespace if loading
+        // precompiled schema because those are built into it. Should be null or "".
+        logger.warn("Root element name '{}' is ignored when used with precompiled DFDL schema.",
+            rootName);
+      }
+      try {
+        dmp.loadSchema(schemaFileURI);
+      } catch (IOException | InvalidParserException e) {
+        throw new CompileFailure(e);

Review Comment:
   Error architecture? 
   
   This loadSchema call needs to happen on every node, and so has the potential (if the loaded binary schema file is no good or mismatches the Daffodil library version) to fail. Is throwing this exception the right thing here or are other steps preferred/necessary?



##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilBatchReader.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.drill.exec.store.daffodil;
+
+import org.apache.daffodil.japi.DataProcessor;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.physical.impl.scan.v3.ManagedReader;
+import org.apache.drill.exec.physical.impl.scan.v3.file.FileDescrip;
+import org.apache.drill.exec.physical.impl.scan.v3.file.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.store.daffodil.schema.DaffodilDataProcessorFactory;
+import org.apache.drill.exec.store.dfs.DrillFileSystem;
+import org.apache.drill.exec.store.dfs.easy.EasySubScan;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Objects;
+
+import static org.apache.drill.exec.store.daffodil.schema.DaffodilDataProcessorFactory.*;
+import static org.apache.drill.exec.store.daffodil.schema.DrillDaffodilSchemaUtils.daffodilDataProcessorToDrillSchema;
+
+public class DaffodilBatchReader implements ManagedReader {
+
+  private static final Logger logger = LoggerFactory.getLogger(DaffodilBatchReader.class);
+  private final RowSetLoader rowSetLoader;
+  private final CustomErrorContext errorContext;
+  private final DaffodilMessageParser dafParser;
+  private final InputStream dataInputStream;
+
+  public DaffodilBatchReader(DaffodilReaderConfig readerConfig, EasySubScan scan,
+      FileSchemaNegotiator negotiator) {
+
+    errorContext = negotiator.parentErrorContext();
+    DaffodilFormatConfig dafConfig = readerConfig.plugin.getConfig();
+
+    String schemaURIString = dafConfig.getSchemaURI(); // "schema/complexArray1.dfdl.xsd";
+    String rootName = dafConfig.getRootName();
+    String rootNamespace = dafConfig.getRootNamespace();
+    boolean validationMode = dafConfig.getValidationMode();
+
+    URI dfdlSchemaURI;
+    try {
+      dfdlSchemaURI = new URI(schemaURIString);
+    } catch (URISyntaxException e) {
+      throw UserException.validationError(e).build(logger);
+    }
+
+    FileDescrip file = negotiator.file();
+    DrillFileSystem fs = file.fileSystem();
+    URI fsSchemaURI = fs.getUri().resolve(dfdlSchemaURI);
+
+    DaffodilDataProcessorFactory dpf = new DaffodilDataProcessorFactory();
+    DataProcessor dp;
+    try {
+      dp = dpf.getDataProcessor(fsSchemaURI, validationMode, rootName, rootNamespace);
+    } catch (CompileFailure e) {
+      throw UserException.dataReadError(e)
+          .message(String.format("Failed to get Daffodil DFDL processor for: %s", fsSchemaURI))
+          .addContext(errorContext).addContext(e.getMessage()).build(logger);
+    }
+    // Create the corresponding Drill schema.
+    // Note: this could be a very large schema. Think of a large complex RDBMS schema,
+    // all of it, hundreds of tables, but all part of the same metadata tree.
+    TupleMetadata drillSchema = daffodilDataProcessorToDrillSchema(dp);
+    // Inform Drill about the schema
+    negotiator.tableSchema(drillSchema, true);
+
+    //
+    // DATA TIME: Next we construct the runtime objects, and open files.
+    //
+    // We get the DaffodilMessageParser, which is a stateful driver for daffodil that
+    // actually does the parsing.
+    rowSetLoader = negotiator.build().writer();
+
+    // We construct the Daffodil InfosetOutputter which the daffodil parser uses to
+    // convert infoset event calls to fill in a Drill row via a rowSetLoader.
+    DaffodilDrillInfosetOutputter outputter = new DaffodilDrillInfosetOutputter(rowSetLoader);
+
+    // Now we can set up the dafParser with the outputter it will drive with
+    // the parser-produced infoset.
+    dafParser = new DaffodilMessageParser(dp); // needs further initialization after this.
+    dafParser.setInfosetOutputter(outputter);
+
+    Path dataPath = file.split().getPath();
+    // Lastly, we open the data stream
+    try {
+      dataInputStream = fs.openPossiblyCompressedStream(dataPath);
+    } catch (IOException e) {
+      throw UserException.dataReadError(e)
+          .message(String.format("Failed to open input file: %s", dataPath.toString()))
+          .addContext(errorContext).addContext(e.getMessage()).build(logger);
+    }
+    // And lastly,... tell daffodil the input data stream.
+    dafParser.setInputStream(dataInputStream);
+  }
+
+  /**
+   * This is the core of actual processing - data movement from Daffodil to Drill.
+   * <p>
+   * If there is space in the batch, and there is data available to parse then this calls the
+   * daffodil parser, which parses data, delivering it to the rowWriter by way of the infoset
+   * outputter.
+   * <p>
+   * Repeats until the rowWriter is full (a batch is full), or there is no more data, or a parse
+   * error ends execution with a throw.
+   * <p>
+   * Validation errors and other warnings are not errors and are logged but do not cause parsing to
+   * fail/throw.
+   *
+   * @return true if there are rows retrieved, false if no rows were retrieved, which means no more
+   *     will ever be retrieved (end of data).
+   * @throws RuntimeException
+   *     on parse errors.
+   */
+  @Override
+  public boolean next() {
+    // Check assumed invariants
+    // We don't know if there is data or not. This could be called on an empty data file.
+    // We DO know that this won't be called if there is no space in the batch for even 1
+    // row.
+    if (dafParser.isEOF()) {
+      return false; // return without even checking for more rows or trying to parse.
+    }
+    while (rowSetLoader.start() && !dafParser.isEOF()) { // we never zero-trip this loop.
+      // the predicate is always true once.
+      dafParser.parse();
+      if (dafParser.isProcessingError()) {
+        assert (Objects.nonNull(dafParser.getDiagnostics()));
+        throw UserException.dataReadError().message(dafParser.getDiagnosticsAsString())
+            .addContext(errorContext).build(logger);
+      }
+      if (dafParser.isValidationError()) {
+        logger.warn(dafParser.getDiagnosticsAsString());

Review Comment:
   Do we need an option here to convert validation errors to fatal?
   
   Will logger.warn be seen by a query user, or is that just for someone dealing with the logs?
   
   Validation errors either should be escalated to fatal, OR they should be visible in the query output display to a user somehow. 
   
   Either way, users will need a mechanism to suppress validation errors that prove to be unavoidable since they could be common place. Nodody wants thousands of warnings about something they can't avoid that doesn't stop parsing and querying the data. 



##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/schema/DaffodilDataProcessorFactory.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.drill.exec.store.daffodil.schema;
+
+import org.apache.daffodil.japi.Compiler;
+import org.apache.daffodil.japi.Daffodil;
+import org.apache.daffodil.japi.DataProcessor;
+import org.apache.daffodil.japi.Diagnostic;
+import org.apache.daffodil.japi.InvalidParserException;
+import org.apache.daffodil.japi.InvalidUsageException;
+import org.apache.daffodil.japi.ProcessorFactory;
+import org.apache.daffodil.japi.ValidationMode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.nio.channels.Channels;
+import java.util.List;
+import java.util.Objects;
+
+/**
+ * Compiles a DFDL schema (mostly for tests) or loads a pre-compiled DFDL schema so that one can
+ * obtain a DataProcessor for use with DaffodilMessageParser.
+ * <p/>
+ * TODO: Needs to use a cache to avoid reloading/recompiling every time.
+ */
+public class DaffodilDataProcessorFactory {
+  // Default constructor is used.
+
+  private static final Logger logger = LoggerFactory.getLogger(DaffodilDataProcessorFactory.class);
+
+  private DataProcessor dp;
+
+  /**
+   * Gets a Daffodil DataProcessor given the necessary arguments to compile or reload it.
+   *
+   * @param schemaFileURI
+   *     pre-compiled dfdl schema (.bin extension) or DFDL schema source (.xsd extension)
+   * @param validationMode
+   *     Use true to request Daffodil built-in 'limited' validation. Use false for no validation.
+   * @param rootName
+   *     Local name of root element of the message. Can be null to use the first element declaration
+   *     of the primary schema file. Ignored if reloading a pre-compiled schema.
+   * @param rootNS
+   *     Namespace URI as a string. Can be null to use the target namespace of the primary schema
+   *     file or if it is unambiguous what element is the rootName. Ignored if reloading a
+   *     pre-compiled schema.
+   * @return the DataProcessor
+   * @throws CompileFailure
+   *     - if schema compilation fails
+   */
+  public DataProcessor getDataProcessor(URI schemaFileURI, boolean validationMode, String rootName,
+      String rootNS)
+      throws CompileFailure {
+
+    DaffodilDataProcessorFactory dmp = new DaffodilDataProcessorFactory();
+    boolean isPrecompiled = schemaFileURI.toString().endsWith(".bin");
+    if (isPrecompiled) {
+      if (Objects.nonNull(rootName) && !rootName.isEmpty()) {
+        // A usage error. You shouldn't supply the name and optionally namespace if loading
+        // precompiled schema because those are built into it. Should be null or "".
+        logger.warn("Root element name '{}' is ignored when used with precompiled DFDL schema.",
+            rootName);
+      }
+      try {
+        dmp.loadSchema(schemaFileURI);
+      } catch (IOException | InvalidParserException e) {
+        throw new CompileFailure(e);
+      }
+      dmp.setupDP(validationMode, null);
+    } else {
+      List<Diagnostic> pfDiags;
+      try {
+        pfDiags = dmp.compileSchema(schemaFileURI, rootName, rootNS);
+      } catch (URISyntaxException | IOException e) {
+        throw new CompileFailure(e);
+      }
+      dmp.setupDP(validationMode, pfDiags);
+    }
+    return dmp.dp;
+  }
+
+  private void loadSchema(URI schemaFileURI) throws IOException, InvalidParserException {
+    Compiler c = Daffodil.compiler();
+    dp = c.reload(Channels.newChannel(schemaFileURI.toURL().openStream()));

Review Comment:
   @cgivre This reload call is the one that has to happen on every drill node. 
   It needs only to happen once for that schema for the life of the JVM. The "dp" object created here can be reused every time that schema is needed to parse more data. The dp (DataProcessor) is a read only (thread safe) data structure. 
   
   As you see, this can throw exceptions, so the question of how those situations should be handled arises. 
   Even if drill perfectly makes the file available to every node for this, that would rule out the IOException due to file not found or access rights, but a user can create a compiled DFDL schema binary file using the wrong version of the Daffodil schema compiler which is a mismatch for the runtime; hence, it is possible for the InvalidParserException to be thrown. 



##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/schema/DaffodilDataProcessorFactory.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.drill.exec.store.daffodil.schema;
+
+import org.apache.daffodil.japi.Compiler;
+import org.apache.daffodil.japi.Daffodil;
+import org.apache.daffodil.japi.DataProcessor;
+import org.apache.daffodil.japi.Diagnostic;
+import org.apache.daffodil.japi.InvalidParserException;
+import org.apache.daffodil.japi.InvalidUsageException;
+import org.apache.daffodil.japi.ProcessorFactory;
+import org.apache.daffodil.japi.ValidationMode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.nio.channels.Channels;
+import java.util.List;
+import java.util.Objects;
+
+/**
+ * Compiles a DFDL schema (mostly for tests) or loads a pre-compiled DFDL schema so that one can
+ * obtain a DataProcessor for use with DaffodilMessageParser.
+ * <p/>
+ * TODO: Needs to use a cache to avoid reloading/recompiling every time.
+ */
+public class DaffodilDataProcessorFactory {
+  // Default constructor is used.
+
+  private static final Logger logger = LoggerFactory.getLogger(DaffodilDataProcessorFactory.class);
+
+  private DataProcessor dp;
+
+  /**
+   * Gets a Daffodil DataProcessor given the necessary arguments to compile or reload it.
+   *
+   * @param schemaFileURI
+   *     pre-compiled dfdl schema (.bin extension) or DFDL schema source (.xsd extension)
+   * @param validationMode
+   *     Use true to request Daffodil built-in 'limited' validation. Use false for no validation.
+   * @param rootName
+   *     Local name of root element of the message. Can be null to use the first element declaration
+   *     of the primary schema file. Ignored if reloading a pre-compiled schema.
+   * @param rootNS
+   *     Namespace URI as a string. Can be null to use the target namespace of the primary schema
+   *     file or if it is unambiguous what element is the rootName. Ignored if reloading a
+   *     pre-compiled schema.
+   * @return the DataProcessor
+   * @throws CompileFailure
+   *     - if schema compilation fails
+   */
+  public DataProcessor getDataProcessor(URI schemaFileURI, boolean validationMode, String rootName,
+      String rootNS)
+      throws CompileFailure {
+
+    DaffodilDataProcessorFactory dmp = new DaffodilDataProcessorFactory();
+    boolean isPrecompiled = schemaFileURI.toString().endsWith(".bin");
+    if (isPrecompiled) {
+      if (Objects.nonNull(rootName) && !rootName.isEmpty()) {
+        // A usage error. You shouldn't supply the name and optionally namespace if loading
+        // precompiled schema because those are built into it. Should be null or "".
+        logger.warn("Root element name '{}' is ignored when used with precompiled DFDL schema.",
+            rootName);
+      }
+      try {
+        dmp.loadSchema(schemaFileURI);
+      } catch (IOException | InvalidParserException e) {
+        throw new CompileFailure(e);
+      }
+      dmp.setupDP(validationMode, null);
+    } else {
+      List<Diagnostic> pfDiags;
+      try {
+        pfDiags = dmp.compileSchema(schemaFileURI, rootName, rootNS);
+      } catch (URISyntaxException | IOException e) {
+        throw new CompileFailure(e);
+      }
+      dmp.setupDP(validationMode, pfDiags);
+    }
+    return dmp.dp;
+  }
+
+  private void loadSchema(URI schemaFileURI) throws IOException, InvalidParserException {
+    Compiler c = Daffodil.compiler();
+    dp = c.reload(Channels.newChannel(schemaFileURI.toURL().openStream()));
+  }
+
+  private List<Diagnostic> compileSchema(URI schemaFileURI, String rootName, String rootNS)
+      throws URISyntaxException, IOException, CompileFailure {
+    Compiler c = Daffodil.compiler();
+    ProcessorFactory pf = c.compileSource(schemaFileURI, rootName, rootNS);
+    List<Diagnostic> pfDiags = pf.getDiagnostics();
+    if (pf.isError()) {
+      pfDiags.forEach(diag -> logger.error(diag.getSomeMessage()));
+      throw new CompileFailure(pfDiags);
+    }
+    dp = pf.onPath("/");
+    return pfDiags; // must be just warnings. If it was errors we would have thrown.
+  }
+
+  /**
+   * Common setup steps used whether or not we reloaded or compiled a DFDL schema.
+   */
+  private void setupDP(boolean validationMode, List<Diagnostic> pfDiags) throws CompileFailure {
+    Objects.requireNonNull(dp); // true because failure to produce a dp throws CompileFailure.
+    if (validationMode) {
+      try {
+        dp = dp.withValidationMode(ValidationMode.Limited);

Review Comment:
   Add comments about why this is ValidationMode.Limited only. (We don't have the schema text, and aren't creating any XML so using the Full Xerces-based validator is not possible.)



-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] DRILL-8474: Add Daffodil Format Plugin (drill)

Posted by "mbeckerle (via GitHub)" <gi...@apache.org>.
mbeckerle commented on PR #2836:
URL: https://github.com/apache/drill/pull/2836#issuecomment-2081164073

   This now passes all the daffodil contrib tests using the published official Daffodil 3.7.0.
   
   It does not yet run in any scalable fashion, but the metadata/data interfacing is complete. 
   
   I would like to squash this to a single commit before merging, and it needs to be tested rebased onto the latest Drill commit. 


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

To unsubscribe, e-mail: dev-unsubscribe@drill.apache.org

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


Re: [PR] WIP: Preliminary Review on adding Daffodil to Drill (drill)

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


##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilBatchReader.java:
##########
@@ -64,64 +69,97 @@ public DaffodilBatchReader (DaffodilReaderConfig readerConfig, EasySubScan scan,
     this.validationMode = formatConfig.getValidationMode();
 
     //
-    // FIXME: Next, a MIRACLE occurs.
+    // FIXME: Where is this config file to be found? And, what is its syntax?

Review Comment:
   No. That didn't help. I'm not looking for user help here. 
   
   A better question is this: where does a junit test put a config file so that the test will read it?
   I was trying to test my config file stuff, which was failing in completely inscrutable ways. 
   Eventually tracked it down to drills insistence on file extensions. 
   
   My frustration is due to the fact that the entire boilerplate of this plugin configuration stuff could be generated from exactly these lines of specification:
   ```
   name daffodil
   schemaURI String null
   dataURI String null
   rootNamespace String ""
   rootName String ""
   validationMode Boolean false
   ```
   
   That's it. The name is "daffodil" and it has these 5 parameters. Spending time debugging just getting to where my code has its 5 parameters has been quite frustrating. 
   
   I have, however, finally got past this and got a first junit test to work end to end. Daffodil constructs drill metadata, Daffodil parses actual data and Drill row is constructed. 
   
   I could use some help to figure out why I can't access things under src/test/resources. I had to use absolute file URLs to get this to work. 
   
   



-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] DRILL-8474: Add Daffodil Format Plugin (drill)

Posted by "cgivre (via GitHub)" <gi...@apache.org>.
cgivre commented on PR #2836:
URL: https://github.com/apache/drill/pull/2836#issuecomment-1880110452

   > > @mbeckerle I had a thought about your TODO list. See inline.
   > > > This is ready for a next review. All the scalar types are now implemented with typed setter calls.
   > > > The prior review comments have all been addressed I believe.
   > > > Remaining things to do include:
   > > > 
   > > > 1. How to get the compiled DFDL schema object so it can be loaded by daffodil out at the distributed Drill nodes.
   > > 
   > > 
   > > I was thinking about this and I remembered something that might be useful. Drill has support for User Defined Functions (UDF) which are written in Java. To add a UDF to Drill, you also have to write some Java classes in a particular way, and include the JARs. Much like the DFDL class files, the UDF JARs must be accessible to all nodes of a Drill cluster.
   > > Additionally, Drill has the capability of adding UDFs dynamically. This feature was added here: #574. Anyway, I wonder if we could use a similar mechanism to load and store the DFDL files so that they are accessible to all Drill nodes. What do you think?
   > 
   > Excellent: So drill has all the machinery, it's just a question of repackaging it so it's available for this usage pattern, which is a bit different from Drill's UDFs, but also very similar.
   > 
   > There are two user scenarios which we can call production and test.
   > 
   > 1. Production: binary compiled DFDL schema file + code jars for Daffodil's own UDFs and "layers" plugins. This should, ideally, cache the compiled schema and not reload it for every query (at every node), but keep the same loaded instance in memory in a persistant JVM image on each node. For large production DFDL schemas this is the only sensible mechanism as it can take minutes to compile large DFDL schemas.
   > 2. Test: on-the-fly centralized compilation of DFDL schema (from a combination of jars and files) to create and cache (to avoid recompiling) the binary compiled DFDL schema file. Then using that compiled binary file, as item 1. For small DFDL schemas this can be fast enough for production use. Ideally, if the DFDL schema is unchanged this would reuse the compiled binary file, but that's an optimization that may not matter much.
   > 
   > Kinds of objects involved are:
   > 
   > * Daffodil plugin code jars
   > * DFDL schema jars
   > * DFDL schema files (just not packaged into a jar)
   > * Daffodil compiled schema binary file
   > * Daffodil config file - parameters, tunables, and options needed at compile time and/or runtime
   > 
   > Code jars: Daffodil provides two extension features for DFDL users - DFDL UDFs and DFDL 'layers' (ex: plug-ins for uudecode, or gunzip algorithms used in part of the data format). Those are ordinary compiled class files in jars, so in all scenarios those jars are needed on the node class path if the DFDL schema uses them. Daffodil dynamically finds and loads these from the classpath in regular Java Service-Provider Interface (SPI) mechanisms.
   > 
   > Schema jars: Daffodil packages DFDL schema files (source files i.e., mySchema.dfdl.xsd) into jar files to allow inter-schema dependencies to be managed using ordinary jar/java-style managed dependencies. Tools like sbt and maven can express the dependencies of one schema on another, grab and pull them together, etc. Daffodil has a resolver so when one schema file referenes another with include/import it searches the class path directories and jars for the files.
   > 
   > Schema jars are only needed centrally when compiling the schema to a binary file. All references to the jar files for inter-schema file references are compiled into the compiled binary file.
   > 
   > It is possible for one DFDL schema 'project' to define a DFDL schema, along with the code for a plugin like a Daffodil UDF or layer. In that case the one jar created is both a code jar and a schema jar. The schema jar aspects are used when the schema is compiled and ignored at Daffodil runtime. The code jar aspects are used at Daffodil run time and ignored at schema compilation time. So such a jar that is both code and schema jar needs to be on the class path in both places, but there's no interaction of the two things.
   > 
   > Binary Compiled Schema File: Centrally, DFDL schemas in files and/or jars are compiled to create a single binary object which can be reloaded in order to actually use the schema to parse/unparse data.
   > 
   > * These binary files are tied to a specific version+build of Daffodil. (They are just a java object serialization of the runtime data structures used by Daffodil).
   > * Once reloaded into a JVM to create a Daffodil DataProcessor object, that object is read-only so thread safe, and can be shared by parse calls happening on many threads.
   > 
   > Daffodil Config File: This contains settings like what warnings to suppress when compiling and/or at runtime, tunables, such as how large to allow a regex match attempt, maximum parsed data size limit, etc. This also is needed both at schema compile and at runtime, as the same file contains parameters for both DFDL schema compile time and runtime.
   
   @mbeckerle Would you want to chat sometime next week and I can walk you through the UDF architecture?  I don't know how relevant it would be, but you'd at least see how things are installed and so forth.


-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] DRILL-2835: Daffodil Feature for Drill (drill)

Posted by "mbeckerle (via GitHub)" <gi...@apache.org>.
mbeckerle commented on PR #2836:
URL: https://github.com/apache/drill/pull/2836#issuecomment-1867102615

   Rebased onto latest Drill master as of 2023-12-21 (force pushed one more time)
   
   Note that this is never going to pass automated tests until the Daffodil release this depends on is official (currently it needs a locally build Daffodil 3.7.0-snapshot, though the main daffodil branch has the changes integrated so any 3.7.0-snapshot build will work.


-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] DRILL-2835: Daffodil Feature for Drill (drill)

Posted by "mbeckerle (via GitHub)" <gi...@apache.org>.
mbeckerle commented on PR #2836:
URL: https://github.com/apache/drill/pull/2836#issuecomment-1867120954

   > > Rebased onto latest Drill master as of 2023-12-21 (force pushed one more time)
   > > Note that this is never going to pass automated tests until the Daffodil release this depends on is official (currently it needs a locally build Daffodil 3.7.0-snapshot, though the main daffodil branch has the changes integrated so any 3.7.0-snapshot build will work.
   > 
   > @mbeckerle This is really great work! Thanks for your persistence on this. Do you have a an ETA on the next Daffodil release?
   
   We could have a Daffodil release in Jan or Feb. There are some Daffodil API cleanups that need to be discussed that would provide better stability for this Drill integration ... we may want to wait for those and update this to use them. 
    


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

To unsubscribe, e-mail: dev-unsubscribe@drill.apache.org

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


Re: [PR] DRILL-8474: Add Daffodil Format Plugin (drill)

Posted by "cgivre (via GitHub)" <gi...@apache.org>.
cgivre commented on PR #2836:
URL: https://github.com/apache/drill/pull/2836#issuecomment-1902751729

   > @cgivre @paul-rogers is there an example of a Drill UDF that is not part of the drill repository tree?
   > 
   > I'd like to understand the mechanisms for distributing any jar files and dependencies of the UDF that drill uses. I can't find any such in the quasi-USFs that are in the Drill tree, because well, since they are part of Drill, and so are their dependencies, this problem doesn't exist.
   
   
   @mbeckerle Here's an example: https://github.com/datadistillr/drill-humanname-functions.    I'm sorry we weren't able to connect last week.  
   
   


-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] DRILL-8474: Add Daffodil Format Plugin (drill)

Posted by "cgivre (via GitHub)" <gi...@apache.org>.
cgivre commented on code in PR #2836:
URL: https://github.com/apache/drill/pull/2836#discussion_r1442278098


##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilDrillInfosetOutputter.java:
##########
@@ -0,0 +1,296 @@
+/*
+ * 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.drill.exec.store.daffodil;
+
+import org.apache.daffodil.runtime1.api.ComplexElementMetadata;
+import org.apache.daffodil.runtime1.api.ElementMetadata;
+import org.apache.daffodil.runtime1.api.InfosetArray;
+import org.apache.daffodil.runtime1.api.InfosetComplexElement;
+import org.apache.daffodil.japi.infoset.InfosetOutputter;
+import org.apache.daffodil.runtime1.api.InfosetSimpleElement;
+import org.apache.daffodil.runtime1.api.PrimitiveType;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.store.daffodil.schema.DrillDaffodilSchemaUtils;
+import org.apache.drill.exec.store.daffodil.schema.DrillDaffodilSchemaVisitor;
+import org.apache.drill.exec.vector.accessor.ArrayWriter;
+import org.apache.drill.exec.vector.accessor.ColumnWriter;
+import org.apache.drill.exec.vector.accessor.ObjectType;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Stack;
+
+/**
+ * Adapts Daffodil parser infoset event calls to Drill writer calls
+ * to fill in Drill data rows.
+ */
+public class DaffodilDrillInfosetOutputter
+    extends InfosetOutputter {
+
+  private boolean isOriginalRoot() {
+    boolean result = currentTupleWriter() == rowSetWriter;
+    if (result)
+      assert(tupleWriterStack.size() == 1);
+    return result;
+  }
+
+  /**
+   * True if the next startComplex call will be for the
+   * DFDL infoset root element whose children are the columns of
+   * the row set.
+   */
+  private boolean isRootElement = true;
+
+  /**
+   * Stack that is used only if we have sub-structures that are not
+   * simple-type fields of the row.
+   */
+  private final Stack<TupleWriter> tupleWriterStack = new Stack<>();
+
+  private final Stack<ArrayWriter> arrayWriterStack = new Stack<>();
+
+  private TupleWriter currentTupleWriter() {
+    return tupleWriterStack.peek();
+  }
+
+  private ArrayWriter currentArrayWriter() {
+    return arrayWriterStack.peek();
+  }
+
+
+  private static final Logger logger = LoggerFactory.getLogger(DaffodilDrillInfosetOutputter.class);
+
+  private DaffodilDrillInfosetOutputter() {} // no default constructor
+
+  private RowSetLoader rowSetWriter;
+
+  public DaffodilDrillInfosetOutputter(RowSetLoader writer) {
+    this.rowSetWriter = writer;
+    this.tupleWriterStack.push(writer);
+  }
+
+  @Override
+  public void reset() {
+    tupleWriterStack.clear();
+    tupleWriterStack.push(rowSetWriter);
+    arrayWriterStack.clear();
+    this.isRootElement = true;
+    checkCleanState();
+  }
+
+  private void checkCleanState() {
+    assert(isOriginalRoot());
+    assert(arrayWriterStack.isEmpty());
+    assert(isRootElement);
+  }
+
+  @Override
+  public void startDocument() {
+    checkCleanState();
+  }
+
+  @Override
+  public void endDocument() {
+    checkCleanState();
+  }
+
+  private String colName(ElementMetadata md) {
+    return DrillDaffodilSchemaVisitor.makeColumnName(md);
+  }
+
+  @Override
+  public void startSimple(InfosetSimpleElement ise) {
+    assert (!isRootElement);
+    ElementMetadata md = ise.metadata();
+    String colName = colName(md);
+    ColumnWriter cw;
+    if (md.isArray()) {
+      // A simple type array
+      assert(!arrayWriterStack.isEmpty());
+      cw = currentArrayWriter().scalar();
+    } else {
+      // A simple element within a map
+      // Note the map itself might be an array
+      // but we don't care about that here.
+      cw = currentTupleWriter().column(colName);
+    }
+    ColumnMetadata cm = cw.schema();
+    assert(cm.isScalar());
+    if (md.isNillable() && ise.isNilled()) {
+      assert cm.isNullable();
+      cw.setNull();
+    } else {
+      convertDaffodilValueToDrillValue(ise, cm, cw);
+    }
+  }
+
+  @Override
+  public void endSimple(InfosetSimpleElement diSimple) {
+    assert (!isRootElement);
+    // do nothing
+  }
+
+  @Override
+  public void startComplex(InfosetComplexElement ce) {
+    ComplexElementMetadata md = ce.metadata();
+    String colName = colName(ce.metadata());
+    if (isRootElement) {
+      assert(isOriginalRoot());
+      // This complex element's corresponds to the root element of the
+      // DFDL schema. We don't treat this as a column of the row set.
+      // Rather, it's children are the columns of the row set.
+      //
+      // If we do nothing at all here, then we'll start getting
+      // even calls for the children.
+      isRootElement = false;
+      return;
+    }
+    if (md.isArray()) {
+      assert(!arrayWriterStack.isEmpty());
+      // FIXME: is this the way to add a complex array child item (i.e., each array item is a map)
+      tupleWriterStack.push(currentArrayWriter().tuple());
+    } else {
+      tupleWriterStack.push(currentTupleWriter().tuple(colName));
+    }
+  }
+
+  @Override
+  public void endComplex(InfosetComplexElement ce) {
+    ComplexElementMetadata md = ce.metadata();
+    if (isOriginalRoot()) {
+      isRootElement = true;
+      // do nothing else. The row gets closed-out in the DaffodilBatchReader.next() method.
+    } else {
+      // it's a map.
+      // We seem to not need to do anything to end the map. No action taken here works.
+      if (md.isArray()) {
+        assert (!arrayWriterStack.isEmpty());
+        currentArrayWriter().save(); // required for map array entries.
+      }
+      tupleWriterStack.pop();
+    }
+  }
+
+  @Override
+  public void startArray(InfosetArray diArray) {
+    ElementMetadata md = diArray.metadata();
+    assert (md.isArray());
+    // DFDL has no notion of an array directly within another array. A named field (map) is necessary
+    // before you can have another array.
+    assert (currentTupleWriter().type() == ObjectType.TUPLE); // parent is a map, or the top level row.
+    String colName = colName(md);
+    TupleWriter enclosingParentTupleWriter = currentTupleWriter();
+    ArrayWriter aw = enclosingParentTupleWriter.array(colName);
+    arrayWriterStack.push(aw);
+  }
+
+  @Override
+  public void endArray(InfosetArray ia) {
+    ElementMetadata md = ia.metadata();
+    assert (md.isArray());
+    assert (!arrayWriterStack.empty());
+    // FIXME: How do we end/close-out an array?
+    // note that each array instance, when the instance is a map, must have
+    // save called after it is written to the array but that happens
+    // in endComplex events since it must be called not once per array, but
+    // once per array item.
+    arrayWriterStack.pop();
+  }
+
+  private void convertDaffodilValueToDrillValue(InfosetSimpleElement ise, ColumnMetadata cm, ColumnWriter cw) {
+    PrimitiveType dafType = ise.metadata().primitiveType();
+    TypeProtos.MinorType drillType = DrillDaffodilSchemaUtils.getDrillDataType(dafType);
+    assert(drillType == cm.type());
+    switch (drillType) {
+    case INT: {
+      //
+      // FIXME: Javadoc for setObject says "primarily for testing"
+      // So how are we supposed to assign the column value then?
+      // Is there a way to get from a ColumnWriter to a typed scalar writer (downcast perhaps?)
+      cw.setObject(ise.getInt());

Review Comment:
   As a part of Drill's outstanding design there are two classes with the same name of `ScalarWriter`.  We also cleverly configured it so that IDEs will pick up the incorrect one. 
   
   </sarcasm>



-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] DRILL-8474: Add Daffodil Format Plugin (drill)

Posted by "cgivre (via GitHub)" <gi...@apache.org>.
cgivre commented on PR #2836:
URL: https://github.com/apache/drill/pull/2836#issuecomment-1890990577

   > > @mbeckerle With respect to style, I tried to reply to that comment, but the thread won't let me. In any event, Drill classes will typically start with the constructor, then have whatever methods are appropriate for the class. The logger creation usually happens before the constructor. I think all of your other classes followed this format, so the one or two that didn't kind of jumped out at me.
   > 
   > @cgivre I believe the style issues are all fixed. The build did not get any codestyle issues.
   
   The issue I was referring to was more around the organization of a few classes.  Usually we'll have the constructor (if present) at the top followed by any class methods.  I think there was a class or two where the constructor was at the bottom or something like that.  In any event, consider the issue resolved.


-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] DRILL-8474: Add Daffodil Format Plugin (drill)

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


##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/schema/DrillDaffodilSchemaVisitor.java:
##########
@@ -0,0 +1,229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.drill.exec.store.daffodil.schema;
+
+import org.apache.daffodil.runtime1.api.ChoiceMetadata;
+import org.apache.daffodil.runtime1.api.ComplexElementMetadata;
+import org.apache.daffodil.runtime1.api.ElementMetadata;
+import org.apache.daffodil.runtime1.api.InfosetSimpleElement;
+import org.apache.daffodil.runtime1.api.MetadataHandler;
+import org.apache.daffodil.runtime1.api.SequenceMetadata;
+import org.apache.daffodil.runtime1.api.SimpleElementMetadata;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.record.metadata.MapBuilder;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Stack;
+
+/**
+ * This class transforms a DFDL/Daffodil schema into a Drill Schema.
+ */
+public class DrillDaffodilSchemaVisitor extends MetadataHandler {
+  private static final Logger logger = LoggerFactory.getLogger(DrillDaffodilSchemaVisitor.class);
+  /**
+   * Unfortunately, SchemaBuilder and MapBuilder, while similar, do not share a base class so we
+   * have a stack of MapBuilders, and when empty we use the SchemaBuilder

Review Comment:
   This is fixed in the latest commit. Created MapBuilderLike interface shared by SchemaBuilder and MapBuilder. I only populated it with the methods I needed. 
   
   The corresponding problem doesn't really occur in the rowWriter area as tupleWriter is the common underlying class used. 



-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] WIP: Preliminary Review on adding Daffodil to Drill (drill)

Posted by "cgivre (via GitHub)" <gi...@apache.org>.
cgivre commented on code in PR #2836:
URL: https://github.com/apache/drill/pull/2836#discussion_r1364795241


##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilBatchReader.java:
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.drill.exec.store.daffodil;
+
+import java.io.InputStream;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+import org.apache.daffodil.japi.DataProcessor;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.physical.impl.scan.v3.ManagedReader;
+import org.apache.drill.exec.physical.impl.scan.v3.file.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.store.daffodil.schema.DaffodilDataProcessorFactory;
+import org.apache.drill.exec.store.dfs.easy.EasySubScan;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.drill.exec.store.daffodil.schema.DrillDaffodilSchemaUtils.daffodilDataProcessorToDrillSchema;
+
+
+public class DaffodilBatchReader implements ManagedReader {
+
+  private static final Logger logger = LoggerFactory.getLogger(DaffodilBatchReader.class);
+  private final DaffodilFormatConfig formatConfig;
+  private final RowSetLoader rowSetLoader;
+  private final CustomErrorContext errorContext;
+  private final DaffodilMessageParser dafParser;
+  private final boolean validationMode;
+
+  private final InputStream dataInputStream;
+
+  static class DaffodilReaderConfig {
+    final DaffodilFormatPlugin plugin;
+    DaffodilReaderConfig(DaffodilFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public DaffodilBatchReader (DaffodilReaderConfig readerConfig, EasySubScan scan, FileSchemaNegotiator negotiator) {
+
+    errorContext = negotiator.parentErrorContext();
+    this.formatConfig = readerConfig.plugin.getConfig();
+
+    this.validationMode = formatConfig.getValidationMode();
+
+    //
+    // FIXME: Next, a MIRACLE occurs.
+    //
+    // We get the dfdlSchemaURI filled in from the query, or a default config location
+    // We get the rootName (or null if not supplied) from the query, or a default config location
+    // We get the rootNamespace (or null if not supplied) from the query, or a default config location
+    // We get the validationMode (true/false) filled in from the query or a default config location
+    // We get the dataInputURI filled in from the query, or from a default config location
+    //
+    // For a first cut, let's just fake it. :-)
+
+    String rootName = null;
+    String rootNamespace = null;
+
+    URI dfdlSchemaURI;
+    URI dataInputURI;
+
+    try {

Review Comment:
   A few things... 
   1.  I added config variables to the config for the `rootName` and `rootNamespace`.  This means that you can set default values in the config or overwrite them in the query. 
   2. What it looks like to me is that we should do the same for the schema URI as well.  
   
   I think the object you're looking for here to access the file system would be the `negotiator.file().fileSystem()` object.  With that object you can access the file system directly either via `Path` or `URI`.   Take a peek at some of the methods available to you there.  
   
   As an example, in the SHP file reader, we do something similar:
   
   https://github.com/apache/drill/blob/2ab46a9411a52f12a0f9acb1144a318059439bc4/contrib/format-esri/src/main/java/org/apache/drill/exec/store/esri/ShpBatchReader.java#L77-L83
   
   
   



##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilBatchReader.java:
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.drill.exec.store.daffodil;
+
+import java.io.InputStream;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+import org.apache.daffodil.japi.DataProcessor;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.physical.impl.scan.v3.ManagedReader;
+import org.apache.drill.exec.physical.impl.scan.v3.file.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.store.daffodil.schema.DaffodilDataProcessorFactory;
+import org.apache.drill.exec.store.dfs.easy.EasySubScan;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.drill.exec.store.daffodil.schema.DrillDaffodilSchemaUtils.daffodilDataProcessorToDrillSchema;
+
+
+public class DaffodilBatchReader implements ManagedReader {
+
+  private static final Logger logger = LoggerFactory.getLogger(DaffodilBatchReader.class);
+  private final DaffodilFormatConfig formatConfig;
+  private final RowSetLoader rowSetLoader;
+  private final CustomErrorContext errorContext;
+  private final DaffodilMessageParser dafParser;
+  private final boolean validationMode;
+
+  private final InputStream dataInputStream;
+
+  static class DaffodilReaderConfig {
+    final DaffodilFormatPlugin plugin;
+    DaffodilReaderConfig(DaffodilFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public DaffodilBatchReader (DaffodilReaderConfig readerConfig, EasySubScan scan, FileSchemaNegotiator negotiator) {
+
+    errorContext = negotiator.parentErrorContext();
+    this.formatConfig = readerConfig.plugin.getConfig();
+
+    this.validationMode = formatConfig.getValidationMode();
+
+    //
+    // FIXME: Next, a MIRACLE occurs.
+    //
+    // We get the dfdlSchemaURI filled in from the query, or a default config location
+    // We get the rootName (or null if not supplied) from the query, or a default config location
+    // We get the rootNamespace (or null if not supplied) from the query, or a default config location
+    // We get the validationMode (true/false) filled in from the query or a default config location
+    // We get the dataInputURI filled in from the query, or from a default config location
+    //
+    // For a first cut, let's just fake it. :-)
+
+    String rootName = null;
+    String rootNamespace = null;
+
+    URI dfdlSchemaURI;
+    URI dataInputURI;
+
+    try {
+      dfdlSchemaURI = new URI("schema/complexArray1.dfdl.xsd");
+      dataInputURI = new URI("data/complexArray1.dat");
+    } catch (URISyntaxException e) {
+      throw UserException.validationError(e)
+          .message("Error retrieving DFDL schema files")
+          .build(logger);
+    }
+
+
+    // given dfdlSchemaURI and validation settings, and rootName, rootNamespace optionally
+    // get the Daffodil DataProcessor (aka parser static information) that we need, and from that
+    // we get the DaffodilMesageParser, which is a stateful driver for daffodil that actually does
+    // parsing.
+    DaffodilDataProcessorFactory dpf = new DaffodilDataProcessorFactory();
+    DataProcessor dp;
+    try {
+      dp = dpf.getDataProcessor(dfdlSchemaURI, true, rootName, rootNamespace);
+    } catch (Exception e) {
+      throw UserException.dataReadError(e)
+          .message(String.format("Failed to get Daffodil DFDL processor for: %s", dfdlSchemaURI.toString()))
+          .addContext(errorContext).addContext(e.getMessage()).build(logger);
+    }
+    // Create the corresponding Drill schema
+    TupleMetadata drillSchema = daffodilDataProcessorToDrillSchema(dp);
+    // Inform Drill about the schema
+    negotiator.tableSchema(drillSchema, true);
+    //
+    // FIXME: Now a MIRACLE occurs. We get the drill row writer (actually a rowSetLoader)??
+    //
+    rowSetLoader = negotiator.build().writer(); // FIXME: is this right?
+
+    // We construct the Daffodil InfosetOutputter which the daffodil parser uses to
+    // convert infoset event calls to fill in a Drill row via a rowSetLoader.
+    DaffodilDrillInfosetOutputter outputter = new DaffodilDrillInfosetOutputter(rowSetLoader);
+    // Now we can setup the dafParser with the outputter it will drive with the parser-produced
+    // infoset.
+    dafParser = new DaffodilMessageParser(dp); // needs further initialization after this.
+    dafParser.setInfosetOutputter(outputter);
+    // Lastly, we open the data stream
+    try {
+      dataInputStream = dataInputURI.toURL().openStream();

Review Comment:
   Ok, I'm not sure why we need to do this.  Drill can get you an input stream of the input file. 
   All you need to do is:
   
   ```java
   dataInputStream = negotiator.file().fileSystem().openPossiblyCompressedStream(negotiator.file().split().getPath());
   ```



-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] WIP: Preliminary Review on adding Daffodil to Drill (drill)

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


##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilBatchReader.java:
##########
@@ -64,64 +69,97 @@ public DaffodilBatchReader (DaffodilReaderConfig readerConfig, EasySubScan scan,
     this.validationMode = formatConfig.getValidationMode();
 
     //
-    // FIXME: Next, a MIRACLE occurs.
+    // FIXME: Where is this config file to be found? And, what is its syntax?
     //
-    // We get the dfdlSchemaURI filled in from the query, or a default config location
-    // We get the rootName (or null if not supplied) from the query, or a default config location
-    // We get the rootNamespace (or null if not supplied) from the query, or a default config location
-    // We get the validationMode (true/false) filled in from the query or a default config location
-    // We get the dataInputURI filled in from the query, or from a default config location
-    //
-    // For a first cut, let's just fake it. :-)
+    // FIXME: How do I arrange for these same things to be overriddable in the query
+    //   or has that already happened before we get these things?

Review Comment:
   Great.  ... but I still don't know where does Drill look for this config file, what is its naming convention and what is its syntax? 



-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] WIP: Preliminary Review on adding Daffodil to Drill (drill)

Posted by "cgivre (via GitHub)" <gi...@apache.org>.
cgivre commented on code in PR #2836:
URL: https://github.com/apache/drill/pull/2836#discussion_r1375364162


##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilBatchReader.java:
##########
@@ -64,64 +69,97 @@ public DaffodilBatchReader (DaffodilReaderConfig readerConfig, EasySubScan scan,
     this.validationMode = formatConfig.getValidationMode();
 
     //
-    // FIXME: Next, a MIRACLE occurs.
+    // FIXME: Where is this config file to be found? And, what is its syntax?
     //
-    // We get the dfdlSchemaURI filled in from the query, or a default config location
-    // We get the rootName (or null if not supplied) from the query, or a default config location
-    // We get the rootNamespace (or null if not supplied) from the query, or a default config location
-    // We get the validationMode (true/false) filled in from the query or a default config location
-    // We get the dataInputURI filled in from the query, or from a default config location
-    //
-    // For a first cut, let's just fake it. :-)
+    // FIXME: How do I arrange for these same things to be overriddable in the query
+    //   or has that already happened before we get these things?

Review Comment:
   This has already happened.  If you need any variables to be configurable by the user, just add them to `DaffodilFormatConfig`. 
   



-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] WIP: Preliminary Review on adding Daffodil to Drill (drill)

Posted by "cgivre (via GitHub)" <gi...@apache.org>.
cgivre commented on code in PR #2836:
URL: https://github.com/apache/drill/pull/2836#discussion_r1375364409


##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilBatchReader.java:
##########
@@ -64,64 +69,97 @@ public DaffodilBatchReader (DaffodilReaderConfig readerConfig, EasySubScan scan,
     this.validationMode = formatConfig.getValidationMode();
 
     //
-    // FIXME: Next, a MIRACLE occurs.
+    // FIXME: Where is this config file to be found? And, what is its syntax?
     //
-    // We get the dfdlSchemaURI filled in from the query, or a default config location
-    // We get the rootName (or null if not supplied) from the query, or a default config location
-    // We get the rootNamespace (or null if not supplied) from the query, or a default config location
-    // We get the validationMode (true/false) filled in from the query or a default config location
-    // We get the dataInputURI filled in from the query, or from a default config location
-    //
-    // For a first cut, let's just fake it. :-)
+    // FIXME: How do I arrange for these same things to be overriddable in the query
+    //   or has that already happened before we get these things?
 
-    String rootName = null;
-    String rootNamespace = null;
+    DaffodilFormatConfig config = readerConfig.plugin.getConfig();
+    boolean validationMode = config.getValidationMode();
+    String dfdlSchemaURIString = config.getSchemaURI(); // "schema/complexArray1.dfdl.xsd";
+    String rootName = config.getRootName();
+    String rootNamespace = config.getRootNamespace();
+    String dataInputURIString = config.getDataURI(); // "data/complexArray1.dat"
 
     URI dfdlSchemaURI;
     URI dataInputURI;
-
     try {
-      dfdlSchemaURI = new URI("schema/complexArray1.dfdl.xsd");
-      dataInputURI = new URI("data/complexArray1.dat");
+      dfdlSchemaURI = new URI(dfdlSchemaURIString);
+      dataInputURI = new URI(dataInputURIString);
     } catch (URISyntaxException e) {
       throw UserException.validationError(e)
-          .message("Error retrieving DFDL schema files")
           .build(logger);
     }
 
+    DrillFileSystem fs = negotiator.file().fileSystem(); // FIXME: nagging me for a trywithresources?
+    URI fsSchemaURI = fs.getUri().resolve(dfdlSchemaURI);
+    URI fsDataURI = fs.getUri().resolve(dataInputURI);
+    Path fsDataPath = new Path(fsDataURI);
 
+    //
+    // METADATA TIME: Obtain Daffodil metadata, build Drill metadata
+    //
     // given dfdlSchemaURI and validation settings, and rootName, rootNamespace optionally
-    // get the Daffodil DataProcessor (aka parser static information) that we need, and from that
-    // we get the DaffodilMesageParser, which is a stateful driver for daffodil that actually does
-    // parsing.
+    // get the Daffodil DataProcessor (aka parser static information) that we need.
+    //
+
+    //
+    // FIXME: resolve this issue about schema loading
+    //
+    // My hope is that this fsSchemaURI can be opened via toURL().openStream(), i.e., I
+    // don't have to call a DrillFileSystem method to open it.
+    // because if I do, that requires me to refactor getProcessor in Daffodil
+    // which has the code to determine whether this is a source xsd and to search classpath
+    // for component schemas, etc.
+    // DFDL schemas are not small. A good example of a schema is one that is 835 files spread
+    // over a rich directory structure spread over 5 jar files which must be searched in
+    // a specific search order (ex: CLASSPATH Order)
+    // Daffodil simply MUST be able to load, via ordinary getClass().getResource(uri) calls,
+    // all the include/import files that are expressed via relative and absolute paths in
+    // the schema files.
+    //
+    // Daffodil also wants a URI here so that it can issue
+    // diagnostics which refer to it.
+    //
+    // If it is a pre-compiled binary schema then the issue is just that getProcessor() caches
+    // these so they're not reloaded over and over for a series of tests.
+    //
+
+
     DaffodilDataProcessorFactory dpf = new DaffodilDataProcessorFactory();
     DataProcessor dp;
     try {
-      dp = dpf.getDataProcessor(dfdlSchemaURI, true, rootName, rootNamespace);
+      dp = dpf.getDataProcessor(fsSchemaURI, validationMode, rootName, rootNamespace);
     } catch (Exception e) {
       throw UserException.dataReadError(e)
-          .message(String.format("Failed to get Daffodil DFDL processor for: %s", dfdlSchemaURI.toString()))
+          .message(String.format("Failed to get Daffodil DFDL processor for: %s", fsSchemaURI))
           .addContext(errorContext).addContext(e.getMessage()).build(logger);
     }
-    // Create the corresponding Drill schema
+    // Create the corresponding Drill schema.
+    // Note: this could be a very large schema. Think of a large complex RDBMS schema,
+    // all of it, hundreds of tables, but all part of the same metadata tree.
     TupleMetadata drillSchema = daffodilDataProcessorToDrillSchema(dp);
     // Inform Drill about the schema
     negotiator.tableSchema(drillSchema, true);
+
     //
-    // FIXME: Now a MIRACLE occurs. We get the drill row writer (actually a rowSetLoader)??
+    // DATA TIME: Next we construct the runtime objects, and open files.
     //
+    // We get the DaffodilMessageParser, which is a stateful driver for daffodil that
+    // actually does the parsing.
     rowSetLoader = negotiator.build().writer(); // FIXME: is this right?
 
     // We construct the Daffodil InfosetOutputter which the daffodil parser uses to
     // convert infoset event calls to fill in a Drill row via a rowSetLoader.
     DaffodilDrillInfosetOutputter outputter = new DaffodilDrillInfosetOutputter(rowSetLoader);
-    // Now we can setup the dafParser with the outputter it will drive with the parser-produced
-    // infoset.
+

Review Comment:
   The `rowSetLoader` is correct. 



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

To unsubscribe, e-mail: dev-unsubscribe@drill.apache.org

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


Re: [PR] WIP: Preliminary Review on adding Daffodil to Drill (drill)

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


##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilBatchReader.java:
##########
@@ -64,64 +69,97 @@ public DaffodilBatchReader (DaffodilReaderConfig readerConfig, EasySubScan scan,
     this.validationMode = formatConfig.getValidationMode();
 
     //
-    // FIXME: Next, a MIRACLE occurs.
+    // FIXME: Where is this config file to be found? And, what is its syntax?
     //
-    // We get the dfdlSchemaURI filled in from the query, or a default config location
-    // We get the rootName (or null if not supplied) from the query, or a default config location
-    // We get the rootNamespace (or null if not supplied) from the query, or a default config location
-    // We get the validationMode (true/false) filled in from the query or a default config location
-    // We get the dataInputURI filled in from the query, or from a default config location
-    //
-    // For a first cut, let's just fake it. :-)
+    // FIXME: How do I arrange for these same things to be overriddable in the query
+    //   or has that already happened before we get these things?
 
-    String rootName = null;
-    String rootNamespace = null;
+    DaffodilFormatConfig config = readerConfig.plugin.getConfig();
+    boolean validationMode = config.getValidationMode();
+    String dfdlSchemaURIString = config.getSchemaURI(); // "schema/complexArray1.dfdl.xsd";
+    String rootName = config.getRootName();
+    String rootNamespace = config.getRootNamespace();
+    String dataInputURIString = config.getDataURI(); // "data/complexArray1.dat"
 
     URI dfdlSchemaURI;
     URI dataInputURI;
-
     try {
-      dfdlSchemaURI = new URI("schema/complexArray1.dfdl.xsd");
-      dataInputURI = new URI("data/complexArray1.dat");
+      dfdlSchemaURI = new URI(dfdlSchemaURIString);
+      dataInputURI = new URI(dataInputURIString);
     } catch (URISyntaxException e) {
       throw UserException.validationError(e)
-          .message("Error retrieving DFDL schema files")
           .build(logger);
     }
 
+    DrillFileSystem fs = negotiator.file().fileSystem(); // FIXME: nagging me for a trywithresources?
+    URI fsSchemaURI = fs.getUri().resolve(dfdlSchemaURI);
+    URI fsDataURI = fs.getUri().resolve(dataInputURI);
+    Path fsDataPath = new Path(fsDataURI);
 
+    //
+    // METADATA TIME: Obtain Daffodil metadata, build Drill metadata
+    //
     // given dfdlSchemaURI and validation settings, and rootName, rootNamespace optionally
-    // get the Daffodil DataProcessor (aka parser static information) that we need, and from that
-    // we get the DaffodilMesageParser, which is a stateful driver for daffodil that actually does
-    // parsing.
+    // get the Daffodil DataProcessor (aka parser static information) that we need.
+    //
+
+    //
+    // FIXME: resolve this issue about schema loading
+    //
+    // My hope is that this fsSchemaURI can be opened via toURL().openStream(), i.e., I
+    // don't have to call a DrillFileSystem method to open it.
+    // because if I do, that requires me to refactor getProcessor in Daffodil
+    // which has the code to determine whether this is a source xsd and to search classpath
+    // for component schemas, etc.
+    // DFDL schemas are not small. A good example of a schema is one that is 835 files spread
+    // over a rich directory structure spread over 5 jar files which must be searched in
+    // a specific search order (ex: CLASSPATH Order)
+    // Daffodil simply MUST be able to load, via ordinary getClass().getResource(uri) calls,
+    // all the include/import files that are expressed via relative and absolute paths in
+    // the schema files.
+    //
+    // Daffodil also wants a URI here so that it can issue
+    // diagnostics which refer to it.
+    //
+    // If it is a pre-compiled binary schema then the issue is just that getProcessor() caches
+    // these so they're not reloaded over and over for a series of tests.

Review Comment:
   Agree. So long as schemas I put in src/test/resources are available that will work short term. 



-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] WIP: Preliminary Review on adding Daffodil to Drill (drill)

Posted by "cgivre (via GitHub)" <gi...@apache.org>.
cgivre commented on code in PR #2836:
URL: https://github.com/apache/drill/pull/2836#discussion_r1364797899


##########
contrib/format-daffodil/src/test/java/org/apache/drill/exec/store/daffodil/TestDaffodilReader.java:
##########
@@ -0,0 +1,652 @@
+/*
+ * 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.drill.exec.store.daffodil;
+
+import org.apache.drill.categories.RowSetTest;
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.rowSet.RowSet;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.test.ClusterFixture;
+import org.apache.drill.test.ClusterTest;
+import org.apache.drill.test.rowSet.RowSetComparison;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.nio.file.Paths;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+
+import static org.apache.drill.test.QueryTestUtil.generateCompressedFile;
+import static org.apache.drill.test.rowSet.RowSetUtilities.mapArray;
+import static org.apache.drill.test.rowSet.RowSetUtilities.objArray;
+import static org.apache.drill.test.rowSet.RowSetUtilities.strArray;
+import static org.junit.Assert.assertEquals;
+
+@Category(RowSetTest.class)
+public class TestDaffodilReader extends ClusterTest {
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    ClusterTest.startCluster(ClusterFixture.builder(dirTestWatcher));
+
+    DaffodilFormatConfig formatConfig = new DaffodilFormatConfig(null, "", "", false);
+
+    // FIXME: What do these things do? specify xml extension file names are somehow significant?
+    cluster.defineFormat("cp", "daffodil", formatConfig);
+    cluster.defineFormat("dfs", "daffodil", formatConfig);
+
+    // FIXME: Do we need this?

Review Comment:
   You'll need this if you want to run tests with a compressed file.  Drill should be able to read a compressed files so it may be worth throwing a test in with a zipped file or something.



-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] WIP: Preliminary Review on adding Daffodil to Drill (drill)

Posted by "cgivre (via GitHub)" <gi...@apache.org>.
cgivre commented on code in PR #2836:
URL: https://github.com/apache/drill/pull/2836#discussion_r1364798604


##########
contrib/pom.xml:
##########
@@ -59,6 +59,7 @@
     <module>format-pcapng</module>
     <module>format-iceberg</module>
     <module>format-deltalake</module>
+    <module>format-daffodil</module>

Review Comment:
   Please keep these in alphabetical order.



-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] WIP: Preliminary Review on adding Daffodil to Drill (drill)

Posted by "cgivre (via GitHub)" <gi...@apache.org>.
cgivre commented on code in PR #2836:
URL: https://github.com/apache/drill/pull/2836#discussion_r1364798270


##########
distribution/src/assemble/component.xml:
##########


Review Comment:
   Please keep these in alphabetical order.



-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] DRILL-8474: Add Daffodil Format Plugin (drill)

Posted by "cgivre (via GitHub)" <gi...@apache.org>.
cgivre commented on PR #2836:
URL: https://github.com/apache/drill/pull/2836#issuecomment-1880109717

   @mbeckerle 
   With respect to style, I tried to reply to that comment, but the thread won't let me.   In any event, Drill classes will typically start with the constructor, then have whatever methods are appropriate for the class.  The logger creation usually happens before the constructor.  I think all of your other classes followed this format, so the one or two that didn't kind of jumped out at me. 


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

To unsubscribe, e-mail: dev-unsubscribe@drill.apache.org

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


Re: [PR] DRILL-8474: Add Daffodil Format Plugin (drill)

Posted by "mbeckerle (via GitHub)" <gi...@apache.org>.
mbeckerle commented on PR #2836:
URL: https://github.com/apache/drill/pull/2836#issuecomment-1877814024

   Let me respond between the paragraphs....
   
   On Tue, Jan 2, 2024 at 11:49 PM Paul Rogers ***@***.***>
   wrote:
   
   > Hi Mike,
   >
   > Just jumping in with a random thought. Drill has accumulated a number of
   > schema systems: Parquet metadata cache, HMS, Drill's own metastore,
   > "provided schema", and now DFDL. All provide ways of defining data: be it
   > Parquet, JSON, CSV or whatever. One can't help but wonder, should some
   > future version try to reduce this variation somewhat? Maybe map all the
   > variations to DFDL? Map DFDL to Drill's own mechanisms?
   >
   > Well we can dream can't we :-)
   
   I can contribute the ideas in
   https://daffodil.apache.org/dev/design-notes/Proposed-DFDL-Standard-Profile.md
   which
   is an effort to restrict the DFDL language so that schemas written in DFDL
   can work more smoothly with Drill, NiFi, Spark, Flink, Beam, etc. etc.
   
   DFDL's data model is too restrictive to be "the model" for Drill since
   Drill wants to query even unstructured data like XML without schema. DFDL's
   data model is targeted only at structured data.
   
   Drill's data model and APIs seem optimized for streaming block-buffered
   top-level rows of data (the EVF API does anyway). Top level row-sets are
   first-class citizens, as are the fields of said rows. Fields containing
   arrays of maps (possibly containing more arrays of maps, and so on deeply
   nested) are not handled uniformly with the same block-buffered "row-like"
   mechanisms. The APIs are similar, but not polymorphic. I suspect that the
   block-buffered data streaming in Drill only happens for top-level rows,
   because there is no test for whether or not you are allowed to create
   another array item like there is a test for creating another row in a
   row-set writer. There is no control inversion where an adapter must give
   back control to Drill in the middle of trying to write an array.
   
   The current Drill/Daffodil interface I've created doesn't cope with
   header-body* files (ex: PCAP which format has a header record, then
   repeating packet records) as it has no way of returning just the body
   records as top level rows. So while there exists a DFDL schema for PCAP,
   you really do want to use a dedicated PCAP Drill adapter which hands back
   rows, not Daffodil which will parse the entire PCAP file into one huge row
   containing a monster sub-array of packets, where each packet is a map
   within the array of maps. This is ok for now as many files where DFDL is
   used are not like PCAP. They are just repeating records of one format with
   no special whole-file header. Eventually we will want to be able to supply
   a path to tell the Drill/Daffodil interface that you only want the packet
   array as the output rows. (This is the unimplemented Daffodil "onPath(...)"
   API feature. We haven't needed this yet for DFDL work in cybersecurity, but
   it was anticipated 10+ years back as essential for data integration.)
   
   
   > Drill uses two kinds of metadata: schema definitions and file metadata
   > used
   > for scan pruning. Schema information could be used at plan time (to
   > provide
   > column types), but certainly at scan time (to "discover" the defined
   > schema.) File metadata is used primarily at plan time to work out how to
   > distribute work.
   
   
   DFDL has zero notion of file metadata. It doesn't know whether data even
   comes from a file or an open TCP socket. Daffodil/DFDL just sees a
   java.io.InputStream.
   The schema it uses for a given file is specified by the API call. Daffodil
   does nothing itself to try to find or identify any schema.
   
   So we're "blank slate" on this issue with DFDL.
   
   
   >
   >
   > A bit of background on scan pruning. Back in the day, it was common to
   > have
   > thousands or millions of files in Hadoop to scan: this was why tools like
   > Drill were distributed: divide and conquer. And, of course, the fastest
   > scan is to skip files that we know can't contain the information we want.
   > File metadata captures this information outside of the files themselves.
   > HMS was the standard solution in the Hadoop days. (Amazon Glue, for S3, is
   > evidently based on HMS.)
   >
   > For example, Drill's Parquet metadata cache, the Drill metastore and HMS
   > all provide both schema and file metadata information. The schema
   > information mainly helped with schema evolution: over time, different
   > files
   > have different sets of columns. File metadata provides information *about*
   > the file, such as the data ranges stored in each file. For Parquet, we
   > might track that '2023-01-Boston.parquet' has data from the
   > office='Boston'
   > range. (So, no use scanning the file for office='Austin'.) And so on.
   >
   > With Hadoop HFS, it was customary to use directory structure as a partial
   > primary index: our file above would live in the /sales/2023/01 directory,
   > for example, and logic chooses the proper set of directories to scan. In
   > Drill, it is up to the user to add crufty conditionals on the path name.
   > In
   > Impala, and other HMS-aware tools, the user just says WHERE order_year =
   > 2023 AND order_month = 1, and HMS tells the tool that the order_year and
   > order_month columns translate to such-and-so directory paths. Would be
   > nice
   > if Drill could provide that feature as well, given the proper file
   > metadata: in this case, the mapping of column names to path directories
   > and
   > file names.
   
   
   The above all makes perfect sense to me, and DFDL schemas are completely
   orthogonal to this.
   If a file naming convention tells *Drill* that it doesn't need to open and
   parse some data using Daffodil, great, then *Drill* will not invoke
   Daffodil to do so.
   
   DFDL/Daffodil doesn't know nor care about this.
   
   
   >
   > Does DFDL provide only schema information? Does it support versioning so
   > that we know that "old.csv" lacks the "version" column, while "new.csv"
   > includes that column? Does it also include the kinds of file metadata
   > mentioned above?
   
   
   DFDL only provides structural schema information.
   
   Data formats do versioning in a wide variety of ways, so DFDL can't take
   any position on how this is done, but many DFDL schemas adapt to multiple
   versions of the data formats they describe based on the existence of
   different fields or values of those fields. This can only work for formats
   where there are data fields that identify the versions.
   
   But nothing based on file metadata.
   
   
   >
   >
   > Or, perhaps DFDL is used in a different context in which the files have a
   > fixed schema and are small in number? This would fit well the "desktop
   > analytics" model that Charles and James suggested is where Drill is now
   > most commonly used.
   
   
   The cybersecurity use case is one of the prime motivators for DFDL work.
   
   Often the cyber gateways are file movers, files arrive spontaneously in
   various locations, and are moved across the cyber boundary.
   The use cases continue to grow in scale, and some people use Apache NiFi
   with DFDL for large scale such file moving.
   
   Unlike Drill, these use cases all parse and then re-serialize the data
   after extensive validation and rule-based filtering.
   
   The same sort of file-metadata based stuff - ex: rules like all the files
   in this directory named X with extension ".dat" use schema S - all applies
   in the cyber-gateway use case.
   
   Apache Daffodil doesn't know anything about this cyber use case however,
   nor anything about data integration. Daffodil is actually a quite narrow
   library. Stays in its lane.
   
   
   >
   >
   > The answers might suggest if DFDL can be the universal data description.
   > or
   > if DFDL applies just to individual file schemas, and Drill would still
   > need
   > a second system to track schema evolution and file metadata for large
   > deployments.
   
   
   Yeah. Drill needs a separate system for this. Not at all a DFDL-specific
   issue.  DFDL/Daffodil take no position on schema evolution.
   
   However, to Daffodil devs, a DFDL schema is basically source code. We keep
   them in git. They have releases. We package them in jars and use managed
   dependency tools to grab them from repositories the same way java code jars
   are grabbed by maven.
   
   One of my concerns about metadata repositories/registries is that they are
   not thought of as configuration management systems. But DFDL schemas are
   certainly large formal objects that require configuration management.
   
   For example, the VMF schema we have is over 180K lines of DFDL "code",
   spread over hundreds of files. It is actually an assembly composed of
   specific versions of 4 different smaller DFDL schemas and the large corpus
   of VMF-specific schema files. There is documentation, analysis reports,
   etc. that go along with it.
   
   So some sort of repository that makes specific schemas available to Drill
   makes sense, but cannot be confused with the configuration management
   system.
   
   I quite literally just got a Maven Central/Sonotype account yesterday so
   that I can push some DFDL schemas up to maven central so they can be reused
   from there via jars.
   
   
   >
   >
   > Further, if DFDL is kind of a stand-alone thing, with its own reader, then
   > we end up with more complexity: the Drill JSON reader and the DFDL JSON
   > reader. Same for CSV, etc. JSON is so complex that we'd find ourselves
   > telling people that the quirks work one way with the native reader,
   > another
   > way with DFDL. Plus, the DFDL readers might not handle file splits the
   > same
   > way,
   
   
   Daffodil knows no concept of "file splits". It doesn't even know about
   files actually. It's just an input byte stream. literally a
   java.io.InputStream.
   
   
   > or support the same set of formats that Drill's other readers support,
   > and so on. It would be nice to separate the idea of schema description
   > from
   > reader implementation, so that DFDL can be used as a source of schema for
   > any arbitrary reader: both at plan and scan times.
   
   
   The DFDL/Drill integration converts DFDL-described data directly to Drill
   with no intermediate form like XML nor JSON. One hop. E.g.,
   
      drillScalaWriter.setInt(daffodilInfosetElement.getInt());
   
   There is no notion of Daffodil "also" reading JSON. You wouldn't parse JSON
   with DFDL typically. You would use a JSON library and hopefully a JSON
   schema that describes the JSON.
   Ditto for XML, Google protocol buffers, Avro, etc.
   
   
   >
   > If DFDL uses its own readers, then we'd need DFDL reader representations in
   
   
   DFDL is a specific reader, this notion of "its own readers" doesn't apply.
   
   
   >
   > Calcite, which would pick up DFDL schemas so that the schemas are reliably
   > serialized out to each node as part of the physical plan. This is
   > possible,
   > but it does send us down the two-readers-for-every-format path.
   
   
   
   >
   
   On the other hand, if DFDL mapped to Drill's existing schema description,
   > then DFDL could be used with our existing readers
   
   
   I don't get "DFDL used with existing readers".... by "with" you mean
   "along-side" or "incorporating"?
   
   
   > and there would be just
   > one schema description sent to readers: Drill's existing provided schema
   > format that EVF can already consume. At present, just a few formats
   > support
   > provided schema in the Calcite layer: CSV for sure, maybe JSON?
   
   
   This is what we need.  The Daffodil/Drill integration walks DFDL metadata
   and creates Drill metadata 100% in advance and this should, I think,
   automatically find its way to all the right places without anything else
   being needed beyond today's Drill behavior.
   
   But besides Drill's metadata the Daffodil execution at each node needs to
   load up the compiled DFDL schema. That object, which can be several
   megabytes of stuff. Needs to find its way out to all the nodes that need
   it.  This I have no idea how we make happen.
   
   
   >
   > Any thoughts on where this kind of thing might evolve with DFDL in the
   > picture?
   >
   > Thanks,
   >
   > - Paul
   >
   >
   > On Tue, Jan 2, 2024 at 8:00 AM Mike Beckerle ***@***.***>
   > wrote:
   >
   > > @cgivre <https://github.com/cgivre> yes, the next architectural-level
   > > issue is how to get a compiled DFDL schema out to everyplace Drill will
   > run
   > > a Daffodil parse. Every one of those JVMs needs to reload it.
   > >
   > > I'll do the various cleanups and such. The one issue I don't know how to
   > > fix is the "typed setter" vs. (set-object) issue, so if you could steer
   > me
   > > in the right direction on that it would help.
   > >
   > > —
   > > Reply to this email directly, view it on GitHub
   > > <https://github.com/apache/drill/pull/2836#issuecomment-1874213780>, or
   > > unsubscribe
   > > <
   > https://github.com/notifications/unsubscribe-auth/AAYZF4MFVRCUYDCKJYSKKYTYMQVLFAVCNFSM6AAAAAA576F7J2VHI2DSMVQWIX3LMV43OSLTON2WKQ3PNVWWK3TUHMYTQNZUGIYTGNZYGA>
   >
   > > .
   > > You are receiving this because you were mentioned.Message ID:
   > > ***@***.***>
   > >
   >
   > —
   > Reply to this email directly, view it on GitHub
   > <https://github.com/apache/drill/pull/2836#issuecomment-1874845274>, or
   > unsubscribe
   > <https://github.com/notifications/unsubscribe-auth/AALUDA4H366DXIG2RATIV4TYMTPLHAVCNFSM6AAAAAA576F7J2VHI2DSMVQWIX3LMV43OSLTON2WKQ3PNVWWK3TUHMYTQNZUHA2DKMRXGQ>
   > .
   > You are receiving this because you were mentioned.Message ID:
   > ***@***.***>
   >
   


-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] DRILL-2835: Daffodil Feature for Drill (drill)

Posted by "mbeckerle (via GitHub)" <gi...@apache.org>.
mbeckerle commented on PR #2836:
URL: https://github.com/apache/drill/pull/2836#issuecomment-1867092071

   This is pretty much working now, in terms of constructing drill metadata from DFDL schemas, and 
   Daffodil delivering data to Drill. 
   
   There were dozens of commits to get here, so I squashed them as they were no longer helpful. 
   
   Obviously more test are needed, but the ones there show nested subrecords working. 
   
   The issues like how schemas get distributed, and how Daffodil gets invoked in parallel by drill are still open.
   


-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] WIP: Preliminary Review on adding Daffodil to Drill (drill)

Posted by "cgivre (via GitHub)" <gi...@apache.org>.
cgivre commented on code in PR #2836:
URL: https://github.com/apache/drill/pull/2836#discussion_r1377002280


##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilBatchReader.java:
##########
@@ -64,64 +69,97 @@ public DaffodilBatchReader (DaffodilReaderConfig readerConfig, EasySubScan scan,
     this.validationMode = formatConfig.getValidationMode();
 
     //
-    // FIXME: Next, a MIRACLE occurs.
+    // FIXME: Where is this config file to be found? And, what is its syntax?

Review Comment:
   Let me take a look and see what I can figure out.  I agree that Drill does have too much boilerplate code for things like this.



-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] WIP: Preliminary Review on adding Daffodil to Drill (drill)

Posted by "cgivre (via GitHub)" <gi...@apache.org>.
cgivre commented on code in PR #2836:
URL: https://github.com/apache/drill/pull/2836#discussion_r1375364309


##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilBatchReader.java:
##########
@@ -64,64 +69,97 @@ public DaffodilBatchReader (DaffodilReaderConfig readerConfig, EasySubScan scan,
     this.validationMode = formatConfig.getValidationMode();
 
     //
-    // FIXME: Next, a MIRACLE occurs.
+    // FIXME: Where is this config file to be found? And, what is its syntax?
     //
-    // We get the dfdlSchemaURI filled in from the query, or a default config location
-    // We get the rootName (or null if not supplied) from the query, or a default config location
-    // We get the rootNamespace (or null if not supplied) from the query, or a default config location
-    // We get the validationMode (true/false) filled in from the query or a default config location
-    // We get the dataInputURI filled in from the query, or from a default config location
-    //
-    // For a first cut, let's just fake it. :-)
+    // FIXME: How do I arrange for these same things to be overriddable in the query
+    //   or has that already happened before we get these things?
 
-    String rootName = null;
-    String rootNamespace = null;
+    DaffodilFormatConfig config = readerConfig.plugin.getConfig();
+    boolean validationMode = config.getValidationMode();
+    String dfdlSchemaURIString = config.getSchemaURI(); // "schema/complexArray1.dfdl.xsd";
+    String rootName = config.getRootName();
+    String rootNamespace = config.getRootNamespace();
+    String dataInputURIString = config.getDataURI(); // "data/complexArray1.dat"
 
     URI dfdlSchemaURI;
     URI dataInputURI;
-
     try {
-      dfdlSchemaURI = new URI("schema/complexArray1.dfdl.xsd");
-      dataInputURI = new URI("data/complexArray1.dat");
+      dfdlSchemaURI = new URI(dfdlSchemaURIString);
+      dataInputURI = new URI(dataInputURIString);
     } catch (URISyntaxException e) {
       throw UserException.validationError(e)
-          .message("Error retrieving DFDL schema files")
           .build(logger);
     }
 
+    DrillFileSystem fs = negotiator.file().fileSystem(); // FIXME: nagging me for a trywithresources?
+    URI fsSchemaURI = fs.getUri().resolve(dfdlSchemaURI);
+    URI fsDataURI = fs.getUri().resolve(dataInputURI);
+    Path fsDataPath = new Path(fsDataURI);
 
+    //
+    // METADATA TIME: Obtain Daffodil metadata, build Drill metadata
+    //
     // given dfdlSchemaURI and validation settings, and rootName, rootNamespace optionally
-    // get the Daffodil DataProcessor (aka parser static information) that we need, and from that
-    // we get the DaffodilMesageParser, which is a stateful driver for daffodil that actually does
-    // parsing.
+    // get the Daffodil DataProcessor (aka parser static information) that we need.
+    //
+
+    //
+    // FIXME: resolve this issue about schema loading
+    //
+    // My hope is that this fsSchemaURI can be opened via toURL().openStream(), i.e., I
+    // don't have to call a DrillFileSystem method to open it.
+    // because if I do, that requires me to refactor getProcessor in Daffodil
+    // which has the code to determine whether this is a source xsd and to search classpath
+    // for component schemas, etc.
+    // DFDL schemas are not small. A good example of a schema is one that is 835 files spread
+    // over a rich directory structure spread over 5 jar files which must be searched in
+    // a specific search order (ex: CLASSPATH Order)
+    // Daffodil simply MUST be able to load, via ordinary getClass().getResource(uri) calls,
+    // all the include/import files that are expressed via relative and absolute paths in
+    // the schema files.
+    //
+    // Daffodil also wants a URI here so that it can issue
+    // diagnostics which refer to it.
+    //
+    // If it is a pre-compiled binary schema then the issue is just that getProcessor() caches
+    // these so they're not reloaded over and over for a series of tests.

Review Comment:
   May I suggest getting the metadata to work first then we can figure out the schema loading problem.  I hope @paul-rogers or @jnturton could weigh in on that when we're ready. 



-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] WIP: Preliminary Review on adding Daffodil to Drill (drill)

Posted by "cgivre (via GitHub)" <gi...@apache.org>.
cgivre commented on code in PR #2836:
URL: https://github.com/apache/drill/pull/2836#discussion_r1375364126


##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilBatchReader.java:
##########
@@ -64,64 +69,97 @@ public DaffodilBatchReader (DaffodilReaderConfig readerConfig, EasySubScan scan,
     this.validationMode = formatConfig.getValidationMode();
 
     //
-    // FIXME: Next, a MIRACLE occurs.
+    // FIXME: Where is this config file to be found? And, what is its syntax?

Review Comment:
   The config file is the `DaffodilFormatConfig` file.  Drill will handle the precedence for you, meaning that if the users defines the variables in a query, that will overwrite the variables stored in the Drill config.  



-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] DRILL-8474: Add Daffodil Format Plugin (drill)

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


##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilDrillInfosetOutputter.java:
##########
@@ -0,0 +1,296 @@
+/*
+ * 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.drill.exec.store.daffodil;
+
+import org.apache.daffodil.runtime1.api.ComplexElementMetadata;
+import org.apache.daffodil.runtime1.api.ElementMetadata;
+import org.apache.daffodil.runtime1.api.InfosetArray;
+import org.apache.daffodil.runtime1.api.InfosetComplexElement;
+import org.apache.daffodil.japi.infoset.InfosetOutputter;
+import org.apache.daffodil.runtime1.api.InfosetSimpleElement;
+import org.apache.daffodil.runtime1.api.PrimitiveType;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.store.daffodil.schema.DrillDaffodilSchemaUtils;
+import org.apache.drill.exec.store.daffodil.schema.DrillDaffodilSchemaVisitor;
+import org.apache.drill.exec.vector.accessor.ArrayWriter;
+import org.apache.drill.exec.vector.accessor.ColumnWriter;
+import org.apache.drill.exec.vector.accessor.ObjectType;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Stack;
+
+/**
+ * Adapts Daffodil parser infoset event calls to Drill writer calls
+ * to fill in Drill data rows.
+ */
+public class DaffodilDrillInfosetOutputter
+    extends InfosetOutputter {
+
+  private boolean isOriginalRoot() {
+    boolean result = currentTupleWriter() == rowSetWriter;
+    if (result)
+      assert(tupleWriterStack.size() == 1);
+    return result;
+  }
+
+  /**
+   * True if the next startComplex call will be for the
+   * DFDL infoset root element whose children are the columns of
+   * the row set.
+   */
+  private boolean isRootElement = true;
+
+  /**
+   * Stack that is used only if we have sub-structures that are not
+   * simple-type fields of the row.
+   */
+  private final Stack<TupleWriter> tupleWriterStack = new Stack<>();
+
+  private final Stack<ArrayWriter> arrayWriterStack = new Stack<>();
+
+  private TupleWriter currentTupleWriter() {
+    return tupleWriterStack.peek();
+  }
+
+  private ArrayWriter currentArrayWriter() {
+    return arrayWriterStack.peek();
+  }
+
+
+  private static final Logger logger = LoggerFactory.getLogger(DaffodilDrillInfosetOutputter.class);
+
+  private DaffodilDrillInfosetOutputter() {} // no default constructor
+
+  private RowSetLoader rowSetWriter;
+
+  public DaffodilDrillInfosetOutputter(RowSetLoader writer) {
+    this.rowSetWriter = writer;
+    this.tupleWriterStack.push(writer);
+  }
+
+  @Override
+  public void reset() {
+    tupleWriterStack.clear();
+    tupleWriterStack.push(rowSetWriter);
+    arrayWriterStack.clear();
+    this.isRootElement = true;
+    checkCleanState();
+  }
+
+  private void checkCleanState() {
+    assert(isOriginalRoot());
+    assert(arrayWriterStack.isEmpty());
+    assert(isRootElement);
+  }
+
+  @Override
+  public void startDocument() {
+    checkCleanState();
+  }
+
+  @Override
+  public void endDocument() {
+    checkCleanState();
+  }
+
+  private String colName(ElementMetadata md) {
+    return DrillDaffodilSchemaVisitor.makeColumnName(md);
+  }
+
+  @Override
+  public void startSimple(InfosetSimpleElement ise) {
+    assert (!isRootElement);
+    ElementMetadata md = ise.metadata();
+    String colName = colName(md);
+    ColumnWriter cw;
+    if (md.isArray()) {
+      // A simple type array
+      assert(!arrayWriterStack.isEmpty());
+      cw = currentArrayWriter().scalar();
+    } else {
+      // A simple element within a map
+      // Note the map itself might be an array
+      // but we don't care about that here.
+      cw = currentTupleWriter().column(colName);
+    }
+    ColumnMetadata cm = cw.schema();
+    assert(cm.isScalar());
+    if (md.isNillable() && ise.isNilled()) {
+      assert cm.isNullable();
+      cw.setNull();
+    } else {
+      convertDaffodilValueToDrillValue(ise, cm, cw);
+    }
+  }
+
+  @Override
+  public void endSimple(InfosetSimpleElement diSimple) {
+    assert (!isRootElement);
+    // do nothing
+  }
+
+  @Override
+  public void startComplex(InfosetComplexElement ce) {
+    ComplexElementMetadata md = ce.metadata();
+    String colName = colName(ce.metadata());
+    if (isRootElement) {
+      assert(isOriginalRoot());
+      // This complex element's corresponds to the root element of the
+      // DFDL schema. We don't treat this as a column of the row set.
+      // Rather, it's children are the columns of the row set.
+      //
+      // If we do nothing at all here, then we'll start getting
+      // even calls for the children.
+      isRootElement = false;
+      return;
+    }
+    if (md.isArray()) {
+      assert(!arrayWriterStack.isEmpty());
+      // FIXME: is this the way to add a complex array child item (i.e., each array item is a map)
+      tupleWriterStack.push(currentArrayWriter().tuple());
+    } else {
+      tupleWriterStack.push(currentTupleWriter().tuple(colName));
+    }
+  }
+
+  @Override
+  public void endComplex(InfosetComplexElement ce) {
+    ComplexElementMetadata md = ce.metadata();
+    if (isOriginalRoot()) {
+      isRootElement = true;
+      // do nothing else. The row gets closed-out in the DaffodilBatchReader.next() method.
+    } else {
+      // it's a map.
+      // We seem to not need to do anything to end the map. No action taken here works.
+      if (md.isArray()) {
+        assert (!arrayWriterStack.isEmpty());
+        currentArrayWriter().save(); // required for map array entries.
+      }
+      tupleWriterStack.pop();
+    }
+  }
+
+  @Override
+  public void startArray(InfosetArray diArray) {
+    ElementMetadata md = diArray.metadata();
+    assert (md.isArray());
+    // DFDL has no notion of an array directly within another array. A named field (map) is necessary
+    // before you can have another array.
+    assert (currentTupleWriter().type() == ObjectType.TUPLE); // parent is a map, or the top level row.
+    String colName = colName(md);
+    TupleWriter enclosingParentTupleWriter = currentTupleWriter();
+    ArrayWriter aw = enclosingParentTupleWriter.array(colName);
+    arrayWriterStack.push(aw);
+  }
+
+  @Override
+  public void endArray(InfosetArray ia) {
+    ElementMetadata md = ia.metadata();
+    assert (md.isArray());
+    assert (!arrayWriterStack.empty());
+    // FIXME: How do we end/close-out an array?
+    // note that each array instance, when the instance is a map, must have
+    // save called after it is written to the array but that happens
+    // in endComplex events since it must be called not once per array, but
+    // once per array item.
+    arrayWriterStack.pop();
+  }
+
+  private void convertDaffodilValueToDrillValue(InfosetSimpleElement ise, ColumnMetadata cm, ColumnWriter cw) {
+    PrimitiveType dafType = ise.metadata().primitiveType();
+    TypeProtos.MinorType drillType = DrillDaffodilSchemaUtils.getDrillDataType(dafType);
+    assert(drillType == cm.type());
+    switch (drillType) {
+    case INT: {
+      //
+      // FIXME: Javadoc for setObject says "primarily for testing"
+      // So how are we supposed to assign the column value then?
+      // Is there a way to get from a ColumnWriter to a typed scalar writer (downcast perhaps?)
+      cw.setObject(ise.getInt());

Review Comment:
   That works. Why did this seem problematic before? Maybe I had the wrong ScalarWriter type.... 
   Anyway, thanks



-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] WIP: Preliminary Review on adding Daffodil to Drill (drill)

Posted by "cgivre (via GitHub)" <gi...@apache.org>.
cgivre commented on code in PR #2836:
URL: https://github.com/apache/drill/pull/2836#discussion_r1364797516


##########
contrib/format-daffodil/src/test/java/org/apache/drill/exec/store/daffodil/TestDaffodilReader.java:
##########
@@ -0,0 +1,652 @@
+/*
+ * 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.drill.exec.store.daffodil;
+
+import org.apache.drill.categories.RowSetTest;
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.rowSet.RowSet;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.test.ClusterFixture;
+import org.apache.drill.test.ClusterTest;
+import org.apache.drill.test.rowSet.RowSetComparison;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.nio.file.Paths;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+
+import static org.apache.drill.test.QueryTestUtil.generateCompressedFile;
+import static org.apache.drill.test.rowSet.RowSetUtilities.mapArray;
+import static org.apache.drill.test.rowSet.RowSetUtilities.objArray;
+import static org.apache.drill.test.rowSet.RowSetUtilities.strArray;
+import static org.junit.Assert.assertEquals;
+
+@Category(RowSetTest.class)
+public class TestDaffodilReader extends ClusterTest {
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    ClusterTest.startCluster(ClusterFixture.builder(dirTestWatcher));
+
+    DaffodilFormatConfig formatConfig = new DaffodilFormatConfig(null, "", "", false);
+
+    // FIXME: What do these things do? specify xml extension file names are somehow significant?
+    cluster.defineFormat("cp", "daffodil", formatConfig);

Review Comment:
   These are equivalent to the default plugins you get in Drill when you first install it.  For your tests, you really only need one.  I'd just go with `dfs`, but it doesn't really matter.



-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] DRILL-2835: Daffodil Feature for Drill (drill)

Posted by "cgivre (via GitHub)" <gi...@apache.org>.
cgivre commented on code in PR #2836:
URL: https://github.com/apache/drill/pull/2836#discussion_r1439055155


##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilBatchReader.java:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.drill.exec.store.daffodil;
+
+import java.io.InputStream;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Objects;
+
+import org.apache.daffodil.japi.DataProcessor;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.physical.impl.scan.v3.ManagedReader;
+import org.apache.drill.exec.physical.impl.scan.v3.file.FileDescrip;
+import org.apache.drill.exec.physical.impl.scan.v3.file.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.store.daffodil.schema.DaffodilDataProcessorFactory;
+import org.apache.drill.exec.store.dfs.DrillFileSystem;
+import org.apache.drill.exec.store.dfs.easy.EasySubScan;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.drill.exec.store.daffodil.schema.DrillDaffodilSchemaUtils.daffodilDataProcessorToDrillSchema;
+
+
+public class DaffodilBatchReader implements ManagedReader {
+
+  private static final Logger logger = LoggerFactory.getLogger(DaffodilBatchReader.class);
+  private final DaffodilFormatConfig dafConfig;
+  private final RowSetLoader rowSetLoader;
+  private final CustomErrorContext errorContext;
+  private final DaffodilMessageParser dafParser;
+  private final InputStream dataInputStream;
+
+  static class DaffodilReaderConfig {
+    final DaffodilFormatPlugin plugin;
+    DaffodilReaderConfig(DaffodilFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public DaffodilBatchReader (DaffodilReaderConfig readerConfig, EasySubScan scan, FileSchemaNegotiator negotiator) {
+
+    errorContext = negotiator.parentErrorContext();
+    this.dafConfig = readerConfig.plugin.getConfig();
+
+    String schemaURIString = dafConfig.getSchemaURI(); // "schema/complexArray1.dfdl.xsd";
+    String rootName = dafConfig.getRootName();
+    String rootNamespace = dafConfig.getRootNamespace();
+    boolean validationMode = dafConfig.getValidationMode();
+
+    URI dfdlSchemaURI;
+    try {
+      dfdlSchemaURI = new URI(schemaURIString);
+    } catch (URISyntaxException e) {
+      throw UserException.validationError(e)
+          .build(logger);
+    }
+
+    FileDescrip file = negotiator.file();
+    DrillFileSystem fs = file.fileSystem();
+    URI fsSchemaURI = fs.getUri().resolve(dfdlSchemaURI);
+
+
+    DaffodilDataProcessorFactory dpf = new DaffodilDataProcessorFactory();
+    DataProcessor dp;
+    try {
+      dp = dpf.getDataProcessor(fsSchemaURI, validationMode, rootName, rootNamespace);
+    } catch (Exception e) {
+      throw UserException.dataReadError(e)
+          .message(String.format("Failed to get Daffodil DFDL processor for: %s", fsSchemaURI))
+          .addContext(errorContext).addContext(e.getMessage()).build(logger);
+    }
+    // Create the corresponding Drill schema.
+    // Note: this could be a very large schema. Think of a large complex RDBMS schema,
+    // all of it, hundreds of tables, but all part of the same metadata tree.
+    TupleMetadata drillSchema = daffodilDataProcessorToDrillSchema(dp);
+    // Inform Drill about the schema
+    negotiator.tableSchema(drillSchema, true);
+
+    //
+    // DATA TIME: Next we construct the runtime objects, and open files.
+    //
+    // We get the DaffodilMessageParser, which is a stateful driver for daffodil that
+    // actually does the parsing.
+    rowSetLoader = negotiator.build().writer();
+
+    // We construct the Daffodil InfosetOutputter which the daffodil parser uses to
+    // convert infoset event calls to fill in a Drill row via a rowSetLoader.
+    DaffodilDrillInfosetOutputter outputter = new DaffodilDrillInfosetOutputter(rowSetLoader);
+
+    // Now we can setup the dafParser with the outputter it will drive with
+    // the parser-produced infoset.
+    dafParser = new DaffodilMessageParser(dp); // needs further initialization after this.
+    dafParser.setInfosetOutputter(outputter);
+
+    Path dataPath = file.split().getPath();
+    // Lastly, we open the data stream
+    try {
+      dataInputStream = fs.openPossiblyCompressedStream(dataPath);
+    } catch (Exception e) {
+      throw UserException.dataReadError(e)
+          .message(String.format("Failed to open input file: %s", dataPath.toString()))
+          .addContext(errorContext).addContext(e.getMessage()).build(logger);
+    }
+    // And lastly,... tell daffodil the input data stream.
+    dafParser.setInputStream(dataInputStream);
+  }
+
+
+  /**
+   * This is the core of actual processing - data movement from Daffodil to Drill.
+   * <p>
+   * If there is space in the batch, and there is data available to parse
+   * then this calls the daffodil parser, which parses data, delivering it to the rowWriter
+   * by way of the infoset outputter.
+   * <p>
+   * Repeats until the rowWriter is full (a batch is full), or there is no more data, or
+   * a parse error ends execution with a throw.
+   * <p>
+   * Validation errors and other warnings are not errors and are logged but do not cause
+   * parsing to fail/throw.
+   * @return true if there are rows retrieved, false if no rows were retrieved, which means
+   * no more will ever be retrieved (end of data).
+   * @throws RuntimeException on parse errors.
+   */
+  @Override
+  public boolean next() {
+    // Check assumed invariants
+    // We don't know if there is data or not. This could be called on an empty data file.
+    // We DO know that this won't be called if there is no space in the batch for even 1
+    // row.
+    if (dafParser.isEOF()) {
+      return false; // return without even checking for more rows or trying to parse.
+    }
+    while (rowSetLoader.start() && !dafParser.isEOF()) { // we never zero-trip this loop.
+      // the predicate is always true once.
+      try {
+        dafParser.parse();
+        if (dafParser.isProcessingError()) {
+          assert(Objects.nonNull(dafParser.getDiagnostics()));
+          throw UserException.dataReadError().message(dafParser.getDiagnosticsAsString())
+              .addContext(errorContext).build(logger);
+        }
+        if (dafParser.isValidationError()) {
+          logger.warn(dafParser.getDiagnosticsAsString());
+          // Note that even if daffodil is set to not validate, validation errors may still occur
+          // from DFDL's "recoverableError" assertions.
+        }
+      } catch (Exception e) {

Review Comment:
   Same comment here.  Do we know what kind(s) of exceptions we may encounter here?



##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilBatchReader.java:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.drill.exec.store.daffodil;
+
+import java.io.InputStream;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Objects;
+
+import org.apache.daffodil.japi.DataProcessor;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.physical.impl.scan.v3.ManagedReader;
+import org.apache.drill.exec.physical.impl.scan.v3.file.FileDescrip;
+import org.apache.drill.exec.physical.impl.scan.v3.file.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.store.daffodil.schema.DaffodilDataProcessorFactory;
+import org.apache.drill.exec.store.dfs.DrillFileSystem;
+import org.apache.drill.exec.store.dfs.easy.EasySubScan;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.drill.exec.store.daffodil.schema.DrillDaffodilSchemaUtils.daffodilDataProcessorToDrillSchema;
+
+
+public class DaffodilBatchReader implements ManagedReader {
+
+  private static final Logger logger = LoggerFactory.getLogger(DaffodilBatchReader.class);
+  private final DaffodilFormatConfig dafConfig;
+  private final RowSetLoader rowSetLoader;
+  private final CustomErrorContext errorContext;
+  private final DaffodilMessageParser dafParser;
+  private final InputStream dataInputStream;
+
+  static class DaffodilReaderConfig {
+    final DaffodilFormatPlugin plugin;
+    DaffodilReaderConfig(DaffodilFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public DaffodilBatchReader (DaffodilReaderConfig readerConfig, EasySubScan scan, FileSchemaNegotiator negotiator) {
+
+    errorContext = negotiator.parentErrorContext();
+    this.dafConfig = readerConfig.plugin.getConfig();
+
+    String schemaURIString = dafConfig.getSchemaURI(); // "schema/complexArray1.dfdl.xsd";
+    String rootName = dafConfig.getRootName();
+    String rootNamespace = dafConfig.getRootNamespace();
+    boolean validationMode = dafConfig.getValidationMode();
+
+    URI dfdlSchemaURI;
+    try {
+      dfdlSchemaURI = new URI(schemaURIString);
+    } catch (URISyntaxException e) {
+      throw UserException.validationError(e)
+          .build(logger);
+    }
+
+    FileDescrip file = negotiator.file();
+    DrillFileSystem fs = file.fileSystem();
+    URI fsSchemaURI = fs.getUri().resolve(dfdlSchemaURI);
+
+
+    DaffodilDataProcessorFactory dpf = new DaffodilDataProcessorFactory();
+    DataProcessor dp;
+    try {
+      dp = dpf.getDataProcessor(fsSchemaURI, validationMode, rootName, rootNamespace);
+    } catch (Exception e) {

Review Comment:
   Can we be more specific here?



##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilDrillInfosetOutputter.java:
##########
@@ -0,0 +1,296 @@
+/*
+ * 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.drill.exec.store.daffodil;
+
+import org.apache.daffodil.runtime1.api.ComplexElementMetadata;
+import org.apache.daffodil.runtime1.api.ElementMetadata;
+import org.apache.daffodil.runtime1.api.InfosetArray;
+import org.apache.daffodil.runtime1.api.InfosetComplexElement;
+import org.apache.daffodil.japi.infoset.InfosetOutputter;
+import org.apache.daffodil.runtime1.api.InfosetSimpleElement;
+import org.apache.daffodil.runtime1.api.PrimitiveType;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.store.daffodil.schema.DrillDaffodilSchemaUtils;
+import org.apache.drill.exec.store.daffodil.schema.DrillDaffodilSchemaVisitor;
+import org.apache.drill.exec.vector.accessor.ArrayWriter;
+import org.apache.drill.exec.vector.accessor.ColumnWriter;
+import org.apache.drill.exec.vector.accessor.ObjectType;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Stack;
+
+/**
+ * Adapts Daffodil parser infoset event calls to Drill writer calls
+ * to fill in Drill data rows.
+ */
+public class DaffodilDrillInfosetOutputter
+    extends InfosetOutputter {
+
+  private boolean isOriginalRoot() {
+    boolean result = currentTupleWriter() == rowSetWriter;
+    if (result)
+      assert(tupleWriterStack.size() == 1);
+    return result;
+  }
+
+  /**
+   * True if the next startComplex call will be for the
+   * DFDL infoset root element whose children are the columns of
+   * the row set.
+   */
+  private boolean isRootElement = true;
+
+  /**
+   * Stack that is used only if we have sub-structures that are not
+   * simple-type fields of the row.
+   */
+  private final Stack<TupleWriter> tupleWriterStack = new Stack<>();
+
+  private final Stack<ArrayWriter> arrayWriterStack = new Stack<>();
+
+  private TupleWriter currentTupleWriter() {
+    return tupleWriterStack.peek();
+  }
+
+  private ArrayWriter currentArrayWriter() {
+    return arrayWriterStack.peek();
+  }
+
+
+  private static final Logger logger = LoggerFactory.getLogger(DaffodilDrillInfosetOutputter.class);
+
+  private DaffodilDrillInfosetOutputter() {} // no default constructor
+
+  private RowSetLoader rowSetWriter;
+
+  public DaffodilDrillInfosetOutputter(RowSetLoader writer) {
+    this.rowSetWriter = writer;
+    this.tupleWriterStack.push(writer);
+  }
+
+  @Override
+  public void reset() {
+    tupleWriterStack.clear();
+    tupleWriterStack.push(rowSetWriter);
+    arrayWriterStack.clear();
+    this.isRootElement = true;
+    checkCleanState();
+  }
+
+  private void checkCleanState() {
+    assert(isOriginalRoot());
+    assert(arrayWriterStack.isEmpty());
+    assert(isRootElement);
+  }
+
+  @Override
+  public void startDocument() {
+    checkCleanState();
+  }
+
+  @Override
+  public void endDocument() {
+    checkCleanState();
+  }
+
+  private String colName(ElementMetadata md) {
+    return DrillDaffodilSchemaVisitor.makeColumnName(md);
+  }
+
+  @Override
+  public void startSimple(InfosetSimpleElement ise) {
+    assert (!isRootElement);
+    ElementMetadata md = ise.metadata();
+    String colName = colName(md);
+    ColumnWriter cw;
+    if (md.isArray()) {
+      // A simple type array
+      assert(!arrayWriterStack.isEmpty());
+      cw = currentArrayWriter().scalar();
+    } else {
+      // A simple element within a map
+      // Note the map itself might be an array
+      // but we don't care about that here.
+      cw = currentTupleWriter().column(colName);
+    }
+    ColumnMetadata cm = cw.schema();
+    assert(cm.isScalar());
+    if (md.isNillable() && ise.isNilled()) {
+      assert cm.isNullable();
+      cw.setNull();
+    } else {
+      convertDaffodilValueToDrillValue(ise, cm, cw);
+    }
+  }
+
+  @Override
+  public void endSimple(InfosetSimpleElement diSimple) {
+    assert (!isRootElement);
+    // do nothing
+  }
+
+  @Override
+  public void startComplex(InfosetComplexElement ce) {
+    ComplexElementMetadata md = ce.metadata();
+    String colName = colName(ce.metadata());
+    if (isRootElement) {
+      assert(isOriginalRoot());
+      // This complex element's corresponds to the root element of the
+      // DFDL schema. We don't treat this as a column of the row set.
+      // Rather, it's children are the columns of the row set.
+      //
+      // If we do nothing at all here, then we'll start getting
+      // even calls for the children.
+      isRootElement = false;
+      return;
+    }
+    if (md.isArray()) {
+      assert(!arrayWriterStack.isEmpty());
+      // FIXME: is this the way to add a complex array child item (i.e., each array item is a map)
+      tupleWriterStack.push(currentArrayWriter().tuple());
+    } else {
+      tupleWriterStack.push(currentTupleWriter().tuple(colName));
+    }
+  }
+
+  @Override
+  public void endComplex(InfosetComplexElement ce) {
+    ComplexElementMetadata md = ce.metadata();
+    if (isOriginalRoot()) {
+      isRootElement = true;
+      // do nothing else. The row gets closed-out in the DaffodilBatchReader.next() method.
+    } else {
+      // it's a map.
+      // We seem to not need to do anything to end the map. No action taken here works.
+      if (md.isArray()) {
+        assert (!arrayWriterStack.isEmpty());
+        currentArrayWriter().save(); // required for map array entries.
+      }
+      tupleWriterStack.pop();
+    }
+  }
+
+  @Override
+  public void startArray(InfosetArray diArray) {
+    ElementMetadata md = diArray.metadata();
+    assert (md.isArray());
+    // DFDL has no notion of an array directly within another array. A named field (map) is necessary
+    // before you can have another array.
+    assert (currentTupleWriter().type() == ObjectType.TUPLE); // parent is a map, or the top level row.
+    String colName = colName(md);
+    TupleWriter enclosingParentTupleWriter = currentTupleWriter();
+    ArrayWriter aw = enclosingParentTupleWriter.array(colName);
+    arrayWriterStack.push(aw);
+  }
+
+  @Override
+  public void endArray(InfosetArray ia) {
+    ElementMetadata md = ia.metadata();
+    assert (md.isArray());
+    assert (!arrayWriterStack.empty());
+    // FIXME: How do we end/close-out an array?
+    // note that each array instance, when the instance is a map, must have
+    // save called after it is written to the array but that happens
+    // in endComplex events since it must be called not once per array, but
+    // once per array item.
+    arrayWriterStack.pop();
+  }
+
+  private void convertDaffodilValueToDrillValue(InfosetSimpleElement ise, ColumnMetadata cm, ColumnWriter cw) {
+    PrimitiveType dafType = ise.metadata().primitiveType();
+    TypeProtos.MinorType drillType = DrillDaffodilSchemaUtils.getDrillDataType(dafType);
+    assert(drillType == cm.type());
+    switch (drillType) {
+    case INT: {
+      //
+      // FIXME: Javadoc for setObject says "primarily for testing"
+      // So how are we supposed to assign the column value then?
+      // Is there a way to get from a ColumnWriter to a typed scalar writer (downcast perhaps?)
+      cw.setObject(ise.getInt());
+      break;
+    }
+    case BIGINT: {
+      cw.setObject(ise.getLong());
+      break;
+    }
+    case SMALLINT: {
+      cw.setObject(ise.getShort());
+      break;
+    }
+    case TINYINT: {
+      cw.setObject(ise.getByte());
+      break;
+    }
+//        .put("UNSIGNEDLONG", TypeProtos.MinorType.UINT8)

Review Comment:
   Please remove... 



##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilMessageParser.java:
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.drill.exec.store.daffodil;
+
+
+import org.apache.daffodil.japi.DataProcessor;
+import org.apache.daffodil.japi.Diagnostic;
+import org.apache.daffodil.japi.ParseResult;
+import org.apache.daffodil.japi.infoset.InfosetOutputter;
+import org.apache.daffodil.japi.io.InputSourceDataInputStream;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.InputStream;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * DFDL Daffodil Streaming message parser
+ * <br/>
+ * You construct this providing a DataProcessor obtained from the
+ * DaffodilDataProcessorFactory.
+ * The DataProcessor contains the compiled DFDL schema, ready to use, as
+ * well as whether validation while parsing has been requested.
+ * <br/>
+ * The DataProcessor object may be shared/reused by multiple threads each of which
+ * has its own copy of this class.
+ * This object is, however, stateful, and must not be shared by multiple threads.
+ * <br/>
+ * You must call setInputStream, and setInfosetOutputter before
+ * you call parse().
+ * The input stream and the InfosetOutputter objects are also private to one thread and are stateful
+ * and owned by this object.
+ * Once you have called setInputStream, you should view the input stream as the private property of
+ * this object.
+ * The parse() will invoke the InfosetOutputter's methods to deliver
+ * parsed data, and it may optionally create diagnostics (obtained via getDiagnostics)
+ * indicating which kind they are via the getIsProcessingError, getIsValidationError.
+ * <br/>
+ * Note that the InfosetOutputter may be called many times before a processing error is detected,
+ * as Daffodil delivers result data incrementally.
+ * <br/>
+ * Validation errors do not affect the InfosetOutputter output calls, but indicate that data was
+ * detected that is invalid.
+ * <br/>
+ * When parse() returns, the parse has ended and one can check for errors/diagnostics.
+ * One can call parse() again if there is still data to consume, which is checked via the
+ * isEOF() method.
+ * <br/>
+ * There are no guarantees about where the input stream is positioned between parse() calls.
+ * In particular, it may not be positioned at the start of the next message, as Daffodil may
+ * have pre-fetched additional bytes from the input stream which it found are not part of the
+ * current infoset, but the next one.
+ * The positioning of the input stream may in fact be somewhere in the middle of a byte,
+ * as Daffodil does not require messages to be of lengths that are in whole byte units.
+ * Hence, once you give the input stream to this object via setInputStream, that input stream is
+ * owned privately by this class for ever after.
+ */
+public class DaffodilMessageParser {
+
+  /**
+   * Constructs the parser using a DataProcessor obtained from
+   * a DaffodilDataProcessorFactory.
+   * @param dp
+   */
+  DaffodilMessageParser(DataProcessor dp) {
+    this.dp = dp;
+  }
+
+  /**
+   * Provide the input stream from which data is to be parsed.
+   * <br/>
+   * This input stream is then owned by this object and becomes part of its state.
+   * <br/>
+   * It is; however, the responsibility of the caller to close this
+   * input stream after the completion of all parse calls.
+   * In particular, if a parse error is considered fatal, then
+   * the caller should close the input stream.
+   * There are advanced error-recovery techniques that may attempt to find
+   * data that can be parsed later in the data stream.
+   * In those cases the input stream would not be closed after a processing error,
+   * but such usage is beyond the scope of this javadoc.
+   * @param inputStream
+   */
+  public void setInputStream(InputStream inputStream) {
+    dis = new InputSourceDataInputStream(inputStream);
+  }
+
+  /**
+   * Provides the InfosetOutputter which will be called to deliver
+   * the Infoset via calls to its methods.
+   * @param outputter
+   */
+  public void setInfosetOutputter(InfosetOutputter outputter) {
+    this.outputter = outputter;
+  }
+
+  /**
+   * Called to pull messages from the data stream.
+   * The message 'Infoset' is delivered by way of calls to the InfosetOutputter's methods.
+   * <br/>
+   * After calling this, one may call getIsProcessingError, getIsValiationError, isEOF, and
+   * getDiagnostics.
+   */
+  public void parse() {
+    if (dis == null)
+      throw new IllegalStateException("Input stream must be provided by setInputStream() call.");
+    if (outputter == null)
+      throw new IllegalStateException("InfosetOutputter must be provided by setInfosetOutputter() call.");
+
+    reset();
+    ParseResult res = dp.parse(dis, outputter);
+    isProcessingError = res.isProcessingError();
+    isValidationError = res.isValidationError();
+    diagnostics = res.getDiagnostics();
+  }
+
+  /**
+   * True if the input stream is known to contain no more data.
+   * If the input stream is a true stream, not a file, then temporary unavailability of data
+   * may cause this call to block until the stream is closed from the other end, or data becomes
+   * available.
+   * <br/>
+   * False if the input stream is at EOF, and no more data can be obtained.
+   * It is an error to call parse() after isEOF has returned true.
+   * @return
+   */
+  public boolean isEOF() {
+    return !dis.hasData();
+  }
+
+  /**
+   * True if the parse() call failed with a processing error.
+   * This indicates that the data was not well-formed and could not be
+   * parsed successfully.
+   * <br/>
+   * It is possible for isProcessingError and isValidationError to both be true.
+   * @return
+   */
+  public boolean isProcessingError() { return isProcessingError; }
+
+  /**
+   * True if a validation error occurred during parsing.
+   * Subsequently to a validation error occurring, parsing may succeed or fail.
+   * after the validation error was detected.
+   * @return
+   */
+  public boolean isValidationError() { return isValidationError; }
+
+  /**
+   * After a parse() call this returns null or a list of 1 or more diagnostics.
+   * <br/>
+   * If isProcessingError or isValidationError are true, then this will contain at least 1
+   * diagnostic.
+   * If both are true this will contain at least 2 diagnostics.
+   * @return
+   */
+  public List<Diagnostic> getDiagnostics() { return diagnostics;  }
+  public String getDiagnosticsAsString() {
+    String result = diagnostics.stream()
+        .map(Diagnostic::getMessage)
+        .collect(Collectors.joining("\n"));
+    return result;
+  }
+
+
+  private static final Logger logger = LoggerFactory.getLogger(DaffodilMessageParser.class);

Review Comment:
   See above comment about Drill style for classes.



##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilBatchReader.java:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.drill.exec.store.daffodil;
+
+import java.io.InputStream;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Objects;
+
+import org.apache.daffodil.japi.DataProcessor;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.physical.impl.scan.v3.ManagedReader;
+import org.apache.drill.exec.physical.impl.scan.v3.file.FileDescrip;
+import org.apache.drill.exec.physical.impl.scan.v3.file.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.store.daffodil.schema.DaffodilDataProcessorFactory;
+import org.apache.drill.exec.store.dfs.DrillFileSystem;
+import org.apache.drill.exec.store.dfs.easy.EasySubScan;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.drill.exec.store.daffodil.schema.DrillDaffodilSchemaUtils.daffodilDataProcessorToDrillSchema;
+
+
+public class DaffodilBatchReader implements ManagedReader {
+
+  private static final Logger logger = LoggerFactory.getLogger(DaffodilBatchReader.class);
+  private final DaffodilFormatConfig dafConfig;
+  private final RowSetLoader rowSetLoader;
+  private final CustomErrorContext errorContext;
+  private final DaffodilMessageParser dafParser;
+  private final InputStream dataInputStream;
+
+  static class DaffodilReaderConfig {
+    final DaffodilFormatPlugin plugin;
+    DaffodilReaderConfig(DaffodilFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public DaffodilBatchReader (DaffodilReaderConfig readerConfig, EasySubScan scan, FileSchemaNegotiator negotiator) {
+
+    errorContext = negotiator.parentErrorContext();
+    this.dafConfig = readerConfig.plugin.getConfig();
+
+    String schemaURIString = dafConfig.getSchemaURI(); // "schema/complexArray1.dfdl.xsd";
+    String rootName = dafConfig.getRootName();
+    String rootNamespace = dafConfig.getRootNamespace();
+    boolean validationMode = dafConfig.getValidationMode();
+
+    URI dfdlSchemaURI;
+    try {
+      dfdlSchemaURI = new URI(schemaURIString);
+    } catch (URISyntaxException e) {
+      throw UserException.validationError(e)
+          .build(logger);
+    }
+
+    FileDescrip file = negotiator.file();
+    DrillFileSystem fs = file.fileSystem();
+    URI fsSchemaURI = fs.getUri().resolve(dfdlSchemaURI);
+
+
+    DaffodilDataProcessorFactory dpf = new DaffodilDataProcessorFactory();
+    DataProcessor dp;
+    try {
+      dp = dpf.getDataProcessor(fsSchemaURI, validationMode, rootName, rootNamespace);
+    } catch (Exception e) {
+      throw UserException.dataReadError(e)
+          .message(String.format("Failed to get Daffodil DFDL processor for: %s", fsSchemaURI))
+          .addContext(errorContext).addContext(e.getMessage()).build(logger);
+    }
+    // Create the corresponding Drill schema.
+    // Note: this could be a very large schema. Think of a large complex RDBMS schema,
+    // all of it, hundreds of tables, but all part of the same metadata tree.
+    TupleMetadata drillSchema = daffodilDataProcessorToDrillSchema(dp);
+    // Inform Drill about the schema
+    negotiator.tableSchema(drillSchema, true);
+
+    //
+    // DATA TIME: Next we construct the runtime objects, and open files.
+    //
+    // We get the DaffodilMessageParser, which is a stateful driver for daffodil that
+    // actually does the parsing.
+    rowSetLoader = negotiator.build().writer();
+
+    // We construct the Daffodil InfosetOutputter which the daffodil parser uses to
+    // convert infoset event calls to fill in a Drill row via a rowSetLoader.
+    DaffodilDrillInfosetOutputter outputter = new DaffodilDrillInfosetOutputter(rowSetLoader);
+
+    // Now we can setup the dafParser with the outputter it will drive with
+    // the parser-produced infoset.
+    dafParser = new DaffodilMessageParser(dp); // needs further initialization after this.
+    dafParser.setInfosetOutputter(outputter);
+
+    Path dataPath = file.split().getPath();
+    // Lastly, we open the data stream
+    try {
+      dataInputStream = fs.openPossiblyCompressedStream(dataPath);
+    } catch (Exception e) {
+      throw UserException.dataReadError(e)
+          .message(String.format("Failed to open input file: %s", dataPath.toString()))
+          .addContext(errorContext).addContext(e.getMessage()).build(logger);
+    }
+    // And lastly,... tell daffodil the input data stream.
+    dafParser.setInputStream(dataInputStream);
+  }
+
+
+  /**
+   * This is the core of actual processing - data movement from Daffodil to Drill.
+   * <p>
+   * If there is space in the batch, and there is data available to parse
+   * then this calls the daffodil parser, which parses data, delivering it to the rowWriter
+   * by way of the infoset outputter.
+   * <p>
+   * Repeats until the rowWriter is full (a batch is full), or there is no more data, or
+   * a parse error ends execution with a throw.
+   * <p>
+   * Validation errors and other warnings are not errors and are logged but do not cause
+   * parsing to fail/throw.
+   * @return true if there are rows retrieved, false if no rows were retrieved, which means
+   * no more will ever be retrieved (end of data).
+   * @throws RuntimeException on parse errors.
+   */
+  @Override
+  public boolean next() {
+    // Check assumed invariants
+    // We don't know if there is data or not. This could be called on an empty data file.
+    // We DO know that this won't be called if there is no space in the batch for even 1
+    // row.
+    if (dafParser.isEOF()) {
+      return false; // return without even checking for more rows or trying to parse.
+    }
+    while (rowSetLoader.start() && !dafParser.isEOF()) { // we never zero-trip this loop.
+      // the predicate is always true once.
+      try {
+        dafParser.parse();
+        if (dafParser.isProcessingError()) {
+          assert(Objects.nonNull(dafParser.getDiagnostics()));
+          throw UserException.dataReadError().message(dafParser.getDiagnosticsAsString())
+              .addContext(errorContext).build(logger);
+        }
+        if (dafParser.isValidationError()) {
+          logger.warn(dafParser.getDiagnosticsAsString());
+          // Note that even if daffodil is set to not validate, validation errors may still occur
+          // from DFDL's "recoverableError" assertions.
+        }
+      } catch (Exception e) {
+        throw UserException.dataReadError(e).message("Error parsing file: " + e.getMessage())
+            .addContext(errorContext).build(logger);
+      }
+      rowSetLoader.save();
+    }
+    int nRows = rowSetLoader.rowCount();

Review Comment:
   Do we need this logic here?



##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilBatchReader.java:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.drill.exec.store.daffodil;
+
+import java.io.InputStream;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Objects;
+
+import org.apache.daffodil.japi.DataProcessor;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.physical.impl.scan.v3.ManagedReader;
+import org.apache.drill.exec.physical.impl.scan.v3.file.FileDescrip;
+import org.apache.drill.exec.physical.impl.scan.v3.file.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.store.daffodil.schema.DaffodilDataProcessorFactory;
+import org.apache.drill.exec.store.dfs.DrillFileSystem;
+import org.apache.drill.exec.store.dfs.easy.EasySubScan;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.drill.exec.store.daffodil.schema.DrillDaffodilSchemaUtils.daffodilDataProcessorToDrillSchema;
+
+
+public class DaffodilBatchReader implements ManagedReader {
+
+  private static final Logger logger = LoggerFactory.getLogger(DaffodilBatchReader.class);
+  private final DaffodilFormatConfig dafConfig;
+  private final RowSetLoader rowSetLoader;
+  private final CustomErrorContext errorContext;
+  private final DaffodilMessageParser dafParser;
+  private final InputStream dataInputStream;
+
+  static class DaffodilReaderConfig {
+    final DaffodilFormatPlugin plugin;
+    DaffodilReaderConfig(DaffodilFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public DaffodilBatchReader (DaffodilReaderConfig readerConfig, EasySubScan scan, FileSchemaNegotiator negotiator) {
+
+    errorContext = negotiator.parentErrorContext();
+    this.dafConfig = readerConfig.plugin.getConfig();
+
+    String schemaURIString = dafConfig.getSchemaURI(); // "schema/complexArray1.dfdl.xsd";
+    String rootName = dafConfig.getRootName();
+    String rootNamespace = dafConfig.getRootNamespace();
+    boolean validationMode = dafConfig.getValidationMode();
+
+    URI dfdlSchemaURI;
+    try {
+      dfdlSchemaURI = new URI(schemaURIString);
+    } catch (URISyntaxException e) {
+      throw UserException.validationError(e)
+          .build(logger);
+    }
+
+    FileDescrip file = negotiator.file();
+    DrillFileSystem fs = file.fileSystem();
+    URI fsSchemaURI = fs.getUri().resolve(dfdlSchemaURI);
+
+
+    DaffodilDataProcessorFactory dpf = new DaffodilDataProcessorFactory();
+    DataProcessor dp;
+    try {
+      dp = dpf.getDataProcessor(fsSchemaURI, validationMode, rootName, rootNamespace);
+    } catch (Exception e) {
+      throw UserException.dataReadError(e)
+          .message(String.format("Failed to get Daffodil DFDL processor for: %s", fsSchemaURI))
+          .addContext(errorContext).addContext(e.getMessage()).build(logger);
+    }
+    // Create the corresponding Drill schema.
+    // Note: this could be a very large schema. Think of a large complex RDBMS schema,
+    // all of it, hundreds of tables, but all part of the same metadata tree.
+    TupleMetadata drillSchema = daffodilDataProcessorToDrillSchema(dp);
+    // Inform Drill about the schema
+    negotiator.tableSchema(drillSchema, true);
+
+    //
+    // DATA TIME: Next we construct the runtime objects, and open files.
+    //
+    // We get the DaffodilMessageParser, which is a stateful driver for daffodil that
+    // actually does the parsing.
+    rowSetLoader = negotiator.build().writer();
+
+    // We construct the Daffodil InfosetOutputter which the daffodil parser uses to
+    // convert infoset event calls to fill in a Drill row via a rowSetLoader.
+    DaffodilDrillInfosetOutputter outputter = new DaffodilDrillInfosetOutputter(rowSetLoader);
+
+    // Now we can setup the dafParser with the outputter it will drive with
+    // the parser-produced infoset.
+    dafParser = new DaffodilMessageParser(dp); // needs further initialization after this.
+    dafParser.setInfosetOutputter(outputter);
+
+    Path dataPath = file.split().getPath();
+    // Lastly, we open the data stream
+    try {
+      dataInputStream = fs.openPossiblyCompressedStream(dataPath);
+    } catch (Exception e) {
+      throw UserException.dataReadError(e)
+          .message(String.format("Failed to open input file: %s", dataPath.toString()))
+          .addContext(errorContext).addContext(e.getMessage()).build(logger);
+    }
+    // And lastly,... tell daffodil the input data stream.
+    dafParser.setInputStream(dataInputStream);
+  }
+
+
+  /**
+   * This is the core of actual processing - data movement from Daffodil to Drill.
+   * <p>
+   * If there is space in the batch, and there is data available to parse
+   * then this calls the daffodil parser, which parses data, delivering it to the rowWriter
+   * by way of the infoset outputter.
+   * <p>
+   * Repeats until the rowWriter is full (a batch is full), or there is no more data, or
+   * a parse error ends execution with a throw.
+   * <p>
+   * Validation errors and other warnings are not errors and are logged but do not cause
+   * parsing to fail/throw.
+   * @return true if there are rows retrieved, false if no rows were retrieved, which means
+   * no more will ever be retrieved (end of data).
+   * @throws RuntimeException on parse errors.
+   */
+  @Override
+  public boolean next() {
+    // Check assumed invariants
+    // We don't know if there is data or not. This could be called on an empty data file.
+    // We DO know that this won't be called if there is no space in the batch for even 1
+    // row.
+    if (dafParser.isEOF()) {
+      return false; // return without even checking for more rows or trying to parse.
+    }
+    while (rowSetLoader.start() && !dafParser.isEOF()) { // we never zero-trip this loop.
+      // the predicate is always true once.
+      try {
+        dafParser.parse();
+        if (dafParser.isProcessingError()) {
+          assert(Objects.nonNull(dafParser.getDiagnostics()));
+          throw UserException.dataReadError().message(dafParser.getDiagnosticsAsString())
+              .addContext(errorContext).build(logger);
+        }
+        if (dafParser.isValidationError()) {
+          logger.warn(dafParser.getDiagnosticsAsString());
+          // Note that even if daffodil is set to not validate, validation errors may still occur
+          // from DFDL's "recoverableError" assertions.
+        }
+      } catch (Exception e) {
+        throw UserException.dataReadError(e).message("Error parsing file: " + e.getMessage())
+            .addContext(errorContext).build(logger);
+      }
+      rowSetLoader.save();
+    }
+    int nRows = rowSetLoader.rowCount();
+    assert nRows > 0; // This cannot be zero. If the parse failed we will have already thrown out of here.
+    return true;
+  }
+
+  @Override
+  public void close() {
+    AutoCloseables.closeSilently(dataInputStream);

Review Comment:
   Do we need to close the Daffodil parser, or is it ok to leave that?



##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilDrillInfosetOutputter.java:
##########
@@ -0,0 +1,296 @@
+/*
+ * 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.drill.exec.store.daffodil;
+
+import org.apache.daffodil.runtime1.api.ComplexElementMetadata;
+import org.apache.daffodil.runtime1.api.ElementMetadata;
+import org.apache.daffodil.runtime1.api.InfosetArray;
+import org.apache.daffodil.runtime1.api.InfosetComplexElement;
+import org.apache.daffodil.japi.infoset.InfosetOutputter;
+import org.apache.daffodil.runtime1.api.InfosetSimpleElement;
+import org.apache.daffodil.runtime1.api.PrimitiveType;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.store.daffodil.schema.DrillDaffodilSchemaUtils;
+import org.apache.drill.exec.store.daffodil.schema.DrillDaffodilSchemaVisitor;
+import org.apache.drill.exec.vector.accessor.ArrayWriter;
+import org.apache.drill.exec.vector.accessor.ColumnWriter;
+import org.apache.drill.exec.vector.accessor.ObjectType;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Stack;
+
+/**
+ * Adapts Daffodil parser infoset event calls to Drill writer calls
+ * to fill in Drill data rows.
+ */
+public class DaffodilDrillInfosetOutputter
+    extends InfosetOutputter {
+
+  private boolean isOriginalRoot() {
+    boolean result = currentTupleWriter() == rowSetWriter;
+    if (result)
+      assert(tupleWriterStack.size() == 1);
+    return result;
+  }
+
+  /**
+   * True if the next startComplex call will be for the
+   * DFDL infoset root element whose children are the columns of
+   * the row set.
+   */
+  private boolean isRootElement = true;
+
+  /**
+   * Stack that is used only if we have sub-structures that are not
+   * simple-type fields of the row.
+   */
+  private final Stack<TupleWriter> tupleWriterStack = new Stack<>();
+
+  private final Stack<ArrayWriter> arrayWriterStack = new Stack<>();
+
+  private TupleWriter currentTupleWriter() {
+    return tupleWriterStack.peek();
+  }
+
+  private ArrayWriter currentArrayWriter() {
+    return arrayWriterStack.peek();
+  }
+
+
+  private static final Logger logger = LoggerFactory.getLogger(DaffodilDrillInfosetOutputter.class);
+
+  private DaffodilDrillInfosetOutputter() {} // no default constructor
+
+  private RowSetLoader rowSetWriter;
+
+  public DaffodilDrillInfosetOutputter(RowSetLoader writer) {
+    this.rowSetWriter = writer;
+    this.tupleWriterStack.push(writer);
+  }
+
+  @Override
+  public void reset() {
+    tupleWriterStack.clear();
+    tupleWriterStack.push(rowSetWriter);
+    arrayWriterStack.clear();
+    this.isRootElement = true;
+    checkCleanState();
+  }
+
+  private void checkCleanState() {
+    assert(isOriginalRoot());
+    assert(arrayWriterStack.isEmpty());
+    assert(isRootElement);
+  }
+
+  @Override
+  public void startDocument() {
+    checkCleanState();
+  }
+
+  @Override
+  public void endDocument() {
+    checkCleanState();
+  }
+
+  private String colName(ElementMetadata md) {
+    return DrillDaffodilSchemaVisitor.makeColumnName(md);
+  }
+
+  @Override
+  public void startSimple(InfosetSimpleElement ise) {
+    assert (!isRootElement);
+    ElementMetadata md = ise.metadata();
+    String colName = colName(md);
+    ColumnWriter cw;
+    if (md.isArray()) {
+      // A simple type array
+      assert(!arrayWriterStack.isEmpty());
+      cw = currentArrayWriter().scalar();
+    } else {
+      // A simple element within a map
+      // Note the map itself might be an array
+      // but we don't care about that here.
+      cw = currentTupleWriter().column(colName);
+    }
+    ColumnMetadata cm = cw.schema();
+    assert(cm.isScalar());
+    if (md.isNillable() && ise.isNilled()) {
+      assert cm.isNullable();
+      cw.setNull();
+    } else {
+      convertDaffodilValueToDrillValue(ise, cm, cw);
+    }
+  }
+
+  @Override
+  public void endSimple(InfosetSimpleElement diSimple) {
+    assert (!isRootElement);
+    // do nothing
+  }
+
+  @Override
+  public void startComplex(InfosetComplexElement ce) {
+    ComplexElementMetadata md = ce.metadata();
+    String colName = colName(ce.metadata());
+    if (isRootElement) {
+      assert(isOriginalRoot());
+      // This complex element's corresponds to the root element of the
+      // DFDL schema. We don't treat this as a column of the row set.
+      // Rather, it's children are the columns of the row set.
+      //
+      // If we do nothing at all here, then we'll start getting
+      // even calls for the children.
+      isRootElement = false;
+      return;
+    }
+    if (md.isArray()) {
+      assert(!arrayWriterStack.isEmpty());
+      // FIXME: is this the way to add a complex array child item (i.e., each array item is a map)
+      tupleWriterStack.push(currentArrayWriter().tuple());
+    } else {
+      tupleWriterStack.push(currentTupleWriter().tuple(colName));
+    }
+  }
+
+  @Override
+  public void endComplex(InfosetComplexElement ce) {
+    ComplexElementMetadata md = ce.metadata();
+    if (isOriginalRoot()) {
+      isRootElement = true;
+      // do nothing else. The row gets closed-out in the DaffodilBatchReader.next() method.
+    } else {
+      // it's a map.
+      // We seem to not need to do anything to end the map. No action taken here works.
+      if (md.isArray()) {
+        assert (!arrayWriterStack.isEmpty());
+        currentArrayWriter().save(); // required for map array entries.
+      }
+      tupleWriterStack.pop();
+    }
+  }
+
+  @Override
+  public void startArray(InfosetArray diArray) {
+    ElementMetadata md = diArray.metadata();
+    assert (md.isArray());
+    // DFDL has no notion of an array directly within another array. A named field (map) is necessary
+    // before you can have another array.
+    assert (currentTupleWriter().type() == ObjectType.TUPLE); // parent is a map, or the top level row.
+    String colName = colName(md);
+    TupleWriter enclosingParentTupleWriter = currentTupleWriter();
+    ArrayWriter aw = enclosingParentTupleWriter.array(colName);
+    arrayWriterStack.push(aw);
+  }
+
+  @Override
+  public void endArray(InfosetArray ia) {
+    ElementMetadata md = ia.metadata();
+    assert (md.isArray());
+    assert (!arrayWriterStack.empty());
+    // FIXME: How do we end/close-out an array?
+    // note that each array instance, when the instance is a map, must have
+    // save called after it is written to the array but that happens
+    // in endComplex events since it must be called not once per array, but
+    // once per array item.
+    arrayWriterStack.pop();
+  }
+
+  private void convertDaffodilValueToDrillValue(InfosetSimpleElement ise, ColumnMetadata cm, ColumnWriter cw) {
+    PrimitiveType dafType = ise.metadata().primitiveType();
+    TypeProtos.MinorType drillType = DrillDaffodilSchemaUtils.getDrillDataType(dafType);
+    assert(drillType == cm.type());
+    switch (drillType) {
+    case INT: {
+      //
+      // FIXME: Javadoc for setObject says "primarily for testing"
+      // So how are we supposed to assign the column value then?
+      // Is there a way to get from a ColumnWriter to a typed scalar writer (downcast perhaps?)
+      cw.setObject(ise.getInt());

Review Comment:
   For these methods here, I seem to recall that Drill has dedicated set methods for each data type.  Is there a reason why you chose the `setObject()`?  I suspect that if there are methods for each data type, we should probably use them since setObject probably does a type check anyway.



##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilDrillInfosetOutputter.java:
##########
@@ -0,0 +1,296 @@
+/*
+ * 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.drill.exec.store.daffodil;
+
+import org.apache.daffodil.runtime1.api.ComplexElementMetadata;
+import org.apache.daffodil.runtime1.api.ElementMetadata;
+import org.apache.daffodil.runtime1.api.InfosetArray;
+import org.apache.daffodil.runtime1.api.InfosetComplexElement;
+import org.apache.daffodil.japi.infoset.InfosetOutputter;
+import org.apache.daffodil.runtime1.api.InfosetSimpleElement;
+import org.apache.daffodil.runtime1.api.PrimitiveType;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.store.daffodil.schema.DrillDaffodilSchemaUtils;
+import org.apache.drill.exec.store.daffodil.schema.DrillDaffodilSchemaVisitor;
+import org.apache.drill.exec.vector.accessor.ArrayWriter;
+import org.apache.drill.exec.vector.accessor.ColumnWriter;
+import org.apache.drill.exec.vector.accessor.ObjectType;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Stack;
+
+/**
+ * Adapts Daffodil parser infoset event calls to Drill writer calls
+ * to fill in Drill data rows.
+ */
+public class DaffodilDrillInfosetOutputter
+    extends InfosetOutputter {
+
+  private boolean isOriginalRoot() {
+    boolean result = currentTupleWriter() == rowSetWriter;

Review Comment:
   Nit:  Please use formatting consistent with Drill's style for this class.  Usually we put the logger at the beginning, then constructors. etc.. 



##########
contrib/format-daffodil/src/test/java/org/apache/drill/exec/store/daffodil/TestDaffodilReader.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.drill.exec.store.daffodil;
+
+import org.apache.drill.categories.RowSetTest;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.rowSet.RowSet;
+import org.apache.drill.exec.physical.rowSet.RowSetReader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.test.ClusterFixture;
+import org.apache.drill.test.ClusterTest;
+import org.apache.drill.test.QueryBuilder;
+import org.apache.drill.test.rowSet.RowSetComparison;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.nio.file.Paths;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+@Category(RowSetTest.class)
+public class TestDaffodilReader extends ClusterTest {
+
+  String schemaURIRoot = "file:///opt/drill/contrib/format-daffodil/src/test/resources/";
+  @BeforeClass
+  public static void setup() throws Exception {
+    // boilerplate call to start test rig
+    ClusterTest.startCluster(ClusterFixture.builder(dirTestWatcher));
+
+    DaffodilFormatConfig formatConfig =
+        new DaffodilFormatConfig(null,
+            "",
+            "",
+            "",
+            false);
+
+    cluster.defineFormat("dfs", "daffodil", formatConfig);
+
+    // Needed to test against compressed files.
+    // Copies data from src/test/resources to the dfs root.
+    dirTestWatcher.copyResourceToRoot(Paths.get("data/"));
+    dirTestWatcher.copyResourceToRoot(Paths.get("schema/"));
+  }
+
+  private String selectRow(String schema, String file) {
+    return "SELECT * FROM table(dfs.`data/" + file + "` " +
+        " (type => 'daffodil'," +
+        " validationMode => 'true', " +
+        " schemaURI => '" + schemaURIRoot + "schema/" + schema + ".dfdl.xsd'," +
+        " rootName => 'row'," +
+        " rootNamespace => null " +
+        "))";
+  }
+
+  /**
+   * This unit test tests a simple data file
+   *
+   * @throws Exception Throw exception if anything goes wrong
+   */
+  @Test
+  public void testSimpleQuery1() throws Exception {
+
+    QueryBuilder qb = client.queryBuilder();
+    QueryBuilder query = qb.sql(selectRow("simple", "data01Int.dat.gz"));
+    RowSet results = query.rowSet();
+    results.print();
+    assertEquals(1, results.rowCount());
+
+    // create the expected metadata and data for this test
+    // metadata first
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .add("col", MinorType.INT)
+        .buildSchema();
+
+    RowSet expected = client.rowSetBuilder(expectedSchema)
+        .addRow(0x00000101) // aka 257
+        .build();
+
+    new RowSetComparison(expected).verifyAndClearAll(results);
+  }
+
+  @Test
+  public void testSimpleQuery2() throws Exception {
+
+    QueryBuilder qb = client.queryBuilder();
+    QueryBuilder query = qb.sql(selectRow("simple","data06Int.dat"));
+    RowSet results = query.rowSet();
+    results.print();
+    assertEquals(6, results.rowCount());
+
+    // create the expected metadata and data for this test
+    // metadata first
+    TupleMetadata expectedSchema = new SchemaBuilder()
+            .add("col", MinorType.INT)
+            .buildSchema();
+
+    RowSet expected = client.rowSetBuilder(expectedSchema)
+            .addRow(0x00000101)
+            .addRow(0x00000102)
+            .addRow(0x00000103)
+            .addRow(0x00000104)
+            .addRow(0x00000105)
+            .addRow(0x00000106)
+            .build();
+
+    new RowSetComparison(expected).verifyAndClearAll(results);
+  }
+
+  @Test
+  public void testComplexQuery1() throws Exception {
+
+    QueryBuilder qb = client.queryBuilder();
+    QueryBuilder query = qb.sql(selectRow("complex1", "data02Int.dat"));
+    RowSet results = query.rowSet();
+    results.print();
+    assertEquals(1, results.rowCount());
+
+    RowSetReader rdr = results.reader();
+    rdr.next();
+    String col = rdr.getAsString();
+    assertEquals("{257, 258}", col);
+    assertFalse(rdr.next());
+    results.clear();
+  }
+
+  @Test
+  public void testComplexQuery2() throws Exception {
+
+    QueryBuilder qb = client.queryBuilder();
+    QueryBuilder query = qb.sql(selectRow("complex1", "data06Int.dat"));
+    RowSet results = query.rowSet();
+    results.print();

Review Comment:
   Please remove output from unit tests when they are ready.



##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilDrillInfosetOutputter.java:
##########
@@ -0,0 +1,296 @@
+/*
+ * 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.drill.exec.store.daffodil;
+
+import org.apache.daffodil.runtime1.api.ComplexElementMetadata;
+import org.apache.daffodil.runtime1.api.ElementMetadata;
+import org.apache.daffodil.runtime1.api.InfosetArray;
+import org.apache.daffodil.runtime1.api.InfosetComplexElement;
+import org.apache.daffodil.japi.infoset.InfosetOutputter;
+import org.apache.daffodil.runtime1.api.InfosetSimpleElement;
+import org.apache.daffodil.runtime1.api.PrimitiveType;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.store.daffodil.schema.DrillDaffodilSchemaUtils;
+import org.apache.drill.exec.store.daffodil.schema.DrillDaffodilSchemaVisitor;
+import org.apache.drill.exec.vector.accessor.ArrayWriter;
+import org.apache.drill.exec.vector.accessor.ColumnWriter;
+import org.apache.drill.exec.vector.accessor.ObjectType;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Stack;
+
+/**
+ * Adapts Daffodil parser infoset event calls to Drill writer calls
+ * to fill in Drill data rows.
+ */
+public class DaffodilDrillInfosetOutputter
+    extends InfosetOutputter {
+
+  private boolean isOriginalRoot() {
+    boolean result = currentTupleWriter() == rowSetWriter;
+    if (result)
+      assert(tupleWriterStack.size() == 1);
+    return result;
+  }
+
+  /**
+   * True if the next startComplex call will be for the
+   * DFDL infoset root element whose children are the columns of
+   * the row set.
+   */
+  private boolean isRootElement = true;
+
+  /**
+   * Stack that is used only if we have sub-structures that are not
+   * simple-type fields of the row.
+   */
+  private final Stack<TupleWriter> tupleWriterStack = new Stack<>();
+
+  private final Stack<ArrayWriter> arrayWriterStack = new Stack<>();
+
+  private TupleWriter currentTupleWriter() {
+    return tupleWriterStack.peek();
+  }
+
+  private ArrayWriter currentArrayWriter() {
+    return arrayWriterStack.peek();
+  }
+
+
+  private static final Logger logger = LoggerFactory.getLogger(DaffodilDrillInfosetOutputter.class);
+
+  private DaffodilDrillInfosetOutputter() {} // no default constructor
+
+  private RowSetLoader rowSetWriter;
+
+  public DaffodilDrillInfosetOutputter(RowSetLoader writer) {
+    this.rowSetWriter = writer;
+    this.tupleWriterStack.push(writer);
+  }
+
+  @Override
+  public void reset() {
+    tupleWriterStack.clear();
+    tupleWriterStack.push(rowSetWriter);
+    arrayWriterStack.clear();
+    this.isRootElement = true;
+    checkCleanState();
+  }
+
+  private void checkCleanState() {
+    assert(isOriginalRoot());
+    assert(arrayWriterStack.isEmpty());
+    assert(isRootElement);
+  }
+
+  @Override
+  public void startDocument() {
+    checkCleanState();
+  }
+
+  @Override
+  public void endDocument() {
+    checkCleanState();
+  }
+
+  private String colName(ElementMetadata md) {
+    return DrillDaffodilSchemaVisitor.makeColumnName(md);
+  }
+
+  @Override
+  public void startSimple(InfosetSimpleElement ise) {
+    assert (!isRootElement);
+    ElementMetadata md = ise.metadata();
+    String colName = colName(md);
+    ColumnWriter cw;
+    if (md.isArray()) {
+      // A simple type array
+      assert(!arrayWriterStack.isEmpty());
+      cw = currentArrayWriter().scalar();
+    } else {
+      // A simple element within a map
+      // Note the map itself might be an array
+      // but we don't care about that here.
+      cw = currentTupleWriter().column(colName);
+    }
+    ColumnMetadata cm = cw.schema();
+    assert(cm.isScalar());
+    if (md.isNillable() && ise.isNilled()) {
+      assert cm.isNullable();
+      cw.setNull();
+    } else {
+      convertDaffodilValueToDrillValue(ise, cm, cw);
+    }
+  }
+
+  @Override
+  public void endSimple(InfosetSimpleElement diSimple) {
+    assert (!isRootElement);
+    // do nothing
+  }
+
+  @Override
+  public void startComplex(InfosetComplexElement ce) {
+    ComplexElementMetadata md = ce.metadata();
+    String colName = colName(ce.metadata());
+    if (isRootElement) {
+      assert(isOriginalRoot());
+      // This complex element's corresponds to the root element of the
+      // DFDL schema. We don't treat this as a column of the row set.
+      // Rather, it's children are the columns of the row set.
+      //
+      // If we do nothing at all here, then we'll start getting
+      // even calls for the children.
+      isRootElement = false;
+      return;
+    }
+    if (md.isArray()) {
+      assert(!arrayWriterStack.isEmpty());
+      // FIXME: is this the way to add a complex array child item (i.e., each array item is a map)
+      tupleWriterStack.push(currentArrayWriter().tuple());
+    } else {
+      tupleWriterStack.push(currentTupleWriter().tuple(colName));
+    }
+  }
+
+  @Override
+  public void endComplex(InfosetComplexElement ce) {
+    ComplexElementMetadata md = ce.metadata();
+    if (isOriginalRoot()) {
+      isRootElement = true;
+      // do nothing else. The row gets closed-out in the DaffodilBatchReader.next() method.
+    } else {
+      // it's a map.
+      // We seem to not need to do anything to end the map. No action taken here works.
+      if (md.isArray()) {
+        assert (!arrayWriterStack.isEmpty());
+        currentArrayWriter().save(); // required for map array entries.
+      }
+      tupleWriterStack.pop();
+    }
+  }
+
+  @Override
+  public void startArray(InfosetArray diArray) {
+    ElementMetadata md = diArray.metadata();
+    assert (md.isArray());
+    // DFDL has no notion of an array directly within another array. A named field (map) is necessary
+    // before you can have another array.
+    assert (currentTupleWriter().type() == ObjectType.TUPLE); // parent is a map, or the top level row.
+    String colName = colName(md);
+    TupleWriter enclosingParentTupleWriter = currentTupleWriter();
+    ArrayWriter aw = enclosingParentTupleWriter.array(colName);
+    arrayWriterStack.push(aw);
+  }
+
+  @Override
+  public void endArray(InfosetArray ia) {
+    ElementMetadata md = ia.metadata();
+    assert (md.isArray());
+    assert (!arrayWriterStack.empty());
+    // FIXME: How do we end/close-out an array?
+    // note that each array instance, when the instance is a map, must have
+    // save called after it is written to the array but that happens
+    // in endComplex events since it must be called not once per array, but
+    // once per array item.
+    arrayWriterStack.pop();
+  }
+
+  private void convertDaffodilValueToDrillValue(InfosetSimpleElement ise, ColumnMetadata cm, ColumnWriter cw) {
+    PrimitiveType dafType = ise.metadata().primitiveType();
+    TypeProtos.MinorType drillType = DrillDaffodilSchemaUtils.getDrillDataType(dafType);
+    assert(drillType == cm.type());
+    switch (drillType) {
+    case INT: {
+      //
+      // FIXME: Javadoc for setObject says "primarily for testing"
+      // So how are we supposed to assign the column value then?
+      // Is there a way to get from a ColumnWriter to a typed scalar writer (downcast perhaps?)
+      cw.setObject(ise.getInt());
+      break;
+    }
+    case BIGINT: {
+      cw.setObject(ise.getLong());
+      break;
+    }
+    case SMALLINT: {
+      cw.setObject(ise.getShort());
+      break;
+    }
+    case TINYINT: {
+      cw.setObject(ise.getByte());
+      break;
+    }
+//        .put("UNSIGNEDLONG", TypeProtos.MinorType.UINT8)
+//        .put("UNSIGNEDINT", TypeProtos.MinorType.UINT4)
+//        .put("UNSIGNEDSHORT", TypeProtos.MinorType.UINT2)
+//        .put("UNSIGNEDBYTE", TypeProtos.MinorType.UINT1)
+//        .put("INTEGER", TypeProtos.MinorType.BIGINT)
+//        .put("NONNEGATIVEINTEGER", TypeProtos.MinorType.BIGINT)
+    case BIT: {
+      cw.setObject(ise.getBoolean());
+      break;
+    }
+//        .put("DATE", TypeProtos.MinorType.DATE) // requires conversion
+//        .put("DATETIME", TypeProtos.MinorType.TIMESTAMP) // requires conversion
+//        .put("DECIMAL", TypeProtos.MinorType.VARDECIMAL) // requires conversion (maybe)
+    case FLOAT8: {
+      cw.setObject(ise.getDouble());
+      break;
+    }
+    case FLOAT4: {
+      cw.setObject(ise.getFloat());
+      break;
+    }
+    case VARBINARY: {
+      cw.setObject(ise.getHexBinary());
+      break;
+    }
+    case VARCHAR: {
+      //
+      // FIXME: VARCHAR is defined in drill as utf8 string.
+      // Is Drill expecting something other than a Java string in this setObject call?
+      // Should we be mapping Daffodil strings to Drill VAR16CHAR type?

Review Comment:
   I believe the answer to this question is yes.  We should map Daffodil strings to Drill `VARCHAR` types.  I wouldn't use the `VAR16CHAR` however.



##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilDrillInfosetOutputter.java:
##########
@@ -0,0 +1,296 @@
+/*
+ * 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.drill.exec.store.daffodil;
+
+import org.apache.daffodil.runtime1.api.ComplexElementMetadata;
+import org.apache.daffodil.runtime1.api.ElementMetadata;
+import org.apache.daffodil.runtime1.api.InfosetArray;
+import org.apache.daffodil.runtime1.api.InfosetComplexElement;
+import org.apache.daffodil.japi.infoset.InfosetOutputter;
+import org.apache.daffodil.runtime1.api.InfosetSimpleElement;
+import org.apache.daffodil.runtime1.api.PrimitiveType;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.store.daffodil.schema.DrillDaffodilSchemaUtils;
+import org.apache.drill.exec.store.daffodil.schema.DrillDaffodilSchemaVisitor;
+import org.apache.drill.exec.vector.accessor.ArrayWriter;
+import org.apache.drill.exec.vector.accessor.ColumnWriter;
+import org.apache.drill.exec.vector.accessor.ObjectType;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Stack;
+
+/**
+ * Adapts Daffodil parser infoset event calls to Drill writer calls
+ * to fill in Drill data rows.
+ */
+public class DaffodilDrillInfosetOutputter
+    extends InfosetOutputter {
+
+  private boolean isOriginalRoot() {
+    boolean result = currentTupleWriter() == rowSetWriter;
+    if (result)
+      assert(tupleWriterStack.size() == 1);
+    return result;
+  }
+
+  /**
+   * True if the next startComplex call will be for the
+   * DFDL infoset root element whose children are the columns of
+   * the row set.
+   */
+  private boolean isRootElement = true;
+
+  /**
+   * Stack that is used only if we have sub-structures that are not
+   * simple-type fields of the row.
+   */
+  private final Stack<TupleWriter> tupleWriterStack = new Stack<>();
+
+  private final Stack<ArrayWriter> arrayWriterStack = new Stack<>();
+
+  private TupleWriter currentTupleWriter() {
+    return tupleWriterStack.peek();
+  }
+
+  private ArrayWriter currentArrayWriter() {
+    return arrayWriterStack.peek();
+  }
+
+
+  private static final Logger logger = LoggerFactory.getLogger(DaffodilDrillInfosetOutputter.class);
+
+  private DaffodilDrillInfosetOutputter() {} // no default constructor
+
+  private RowSetLoader rowSetWriter;
+
+  public DaffodilDrillInfosetOutputter(RowSetLoader writer) {
+    this.rowSetWriter = writer;
+    this.tupleWriterStack.push(writer);
+  }
+
+  @Override
+  public void reset() {
+    tupleWriterStack.clear();
+    tupleWriterStack.push(rowSetWriter);
+    arrayWriterStack.clear();
+    this.isRootElement = true;
+    checkCleanState();
+  }
+
+  private void checkCleanState() {
+    assert(isOriginalRoot());
+    assert(arrayWriterStack.isEmpty());
+    assert(isRootElement);
+  }
+
+  @Override
+  public void startDocument() {
+    checkCleanState();
+  }
+
+  @Override
+  public void endDocument() {
+    checkCleanState();
+  }
+
+  private String colName(ElementMetadata md) {
+    return DrillDaffodilSchemaVisitor.makeColumnName(md);
+  }
+
+  @Override
+  public void startSimple(InfosetSimpleElement ise) {
+    assert (!isRootElement);
+    ElementMetadata md = ise.metadata();
+    String colName = colName(md);
+    ColumnWriter cw;
+    if (md.isArray()) {
+      // A simple type array
+      assert(!arrayWriterStack.isEmpty());
+      cw = currentArrayWriter().scalar();
+    } else {
+      // A simple element within a map
+      // Note the map itself might be an array
+      // but we don't care about that here.
+      cw = currentTupleWriter().column(colName);
+    }
+    ColumnMetadata cm = cw.schema();
+    assert(cm.isScalar());
+    if (md.isNillable() && ise.isNilled()) {
+      assert cm.isNullable();
+      cw.setNull();
+    } else {
+      convertDaffodilValueToDrillValue(ise, cm, cw);
+    }
+  }
+
+  @Override
+  public void endSimple(InfosetSimpleElement diSimple) {
+    assert (!isRootElement);
+    // do nothing
+  }
+
+  @Override
+  public void startComplex(InfosetComplexElement ce) {
+    ComplexElementMetadata md = ce.metadata();
+    String colName = colName(ce.metadata());
+    if (isRootElement) {
+      assert(isOriginalRoot());
+      // This complex element's corresponds to the root element of the
+      // DFDL schema. We don't treat this as a column of the row set.
+      // Rather, it's children are the columns of the row set.
+      //
+      // If we do nothing at all here, then we'll start getting
+      // even calls for the children.
+      isRootElement = false;
+      return;
+    }
+    if (md.isArray()) {
+      assert(!arrayWriterStack.isEmpty());
+      // FIXME: is this the way to add a complex array child item (i.e., each array item is a map)
+      tupleWriterStack.push(currentArrayWriter().tuple());
+    } else {
+      tupleWriterStack.push(currentTupleWriter().tuple(colName));
+    }
+  }
+
+  @Override
+  public void endComplex(InfosetComplexElement ce) {
+    ComplexElementMetadata md = ce.metadata();
+    if (isOriginalRoot()) {
+      isRootElement = true;
+      // do nothing else. The row gets closed-out in the DaffodilBatchReader.next() method.
+    } else {
+      // it's a map.
+      // We seem to not need to do anything to end the map. No action taken here works.
+      if (md.isArray()) {
+        assert (!arrayWriterStack.isEmpty());
+        currentArrayWriter().save(); // required for map array entries.
+      }
+      tupleWriterStack.pop();
+    }
+  }
+
+  @Override
+  public void startArray(InfosetArray diArray) {
+    ElementMetadata md = diArray.metadata();
+    assert (md.isArray());
+    // DFDL has no notion of an array directly within another array. A named field (map) is necessary
+    // before you can have another array.
+    assert (currentTupleWriter().type() == ObjectType.TUPLE); // parent is a map, or the top level row.
+    String colName = colName(md);
+    TupleWriter enclosingParentTupleWriter = currentTupleWriter();
+    ArrayWriter aw = enclosingParentTupleWriter.array(colName);
+    arrayWriterStack.push(aw);
+  }
+
+  @Override
+  public void endArray(InfosetArray ia) {
+    ElementMetadata md = ia.metadata();
+    assert (md.isArray());
+    assert (!arrayWriterStack.empty());
+    // FIXME: How do we end/close-out an array?
+    // note that each array instance, when the instance is a map, must have
+    // save called after it is written to the array but that happens
+    // in endComplex events since it must be called not once per array, but
+    // once per array item.
+    arrayWriterStack.pop();
+  }
+
+  private void convertDaffodilValueToDrillValue(InfosetSimpleElement ise, ColumnMetadata cm, ColumnWriter cw) {
+    PrimitiveType dafType = ise.metadata().primitiveType();
+    TypeProtos.MinorType drillType = DrillDaffodilSchemaUtils.getDrillDataType(dafType);
+    assert(drillType == cm.type());
+    switch (drillType) {
+    case INT: {
+      //
+      // FIXME: Javadoc for setObject says "primarily for testing"
+      // So how are we supposed to assign the column value then?
+      // Is there a way to get from a ColumnWriter to a typed scalar writer (downcast perhaps?)
+      cw.setObject(ise.getInt());
+      break;
+    }
+    case BIGINT: {
+      cw.setObject(ise.getLong());
+      break;
+    }
+    case SMALLINT: {
+      cw.setObject(ise.getShort());
+      break;
+    }
+    case TINYINT: {
+      cw.setObject(ise.getByte());
+      break;
+    }
+//        .put("UNSIGNEDLONG", TypeProtos.MinorType.UINT8)
+//        .put("UNSIGNEDINT", TypeProtos.MinorType.UINT4)
+//        .put("UNSIGNEDSHORT", TypeProtos.MinorType.UINT2)
+//        .put("UNSIGNEDBYTE", TypeProtos.MinorType.UINT1)
+//        .put("INTEGER", TypeProtos.MinorType.BIGINT)
+//        .put("NONNEGATIVEINTEGER", TypeProtos.MinorType.BIGINT)
+    case BIT: {
+      cw.setObject(ise.getBoolean());
+      break;
+    }
+//        .put("DATE", TypeProtos.MinorType.DATE) // requires conversion
+//        .put("DATETIME", TypeProtos.MinorType.TIMESTAMP) // requires conversion
+//        .put("DECIMAL", TypeProtos.MinorType.VARDECIMAL) // requires conversion (maybe)
+    case FLOAT8: {
+      cw.setObject(ise.getDouble());
+      break;
+    }
+    case FLOAT4: {
+      cw.setObject(ise.getFloat());
+      break;
+    }
+    case VARBINARY: {
+      cw.setObject(ise.getHexBinary());
+      break;
+    }
+    case VARCHAR: {
+      //
+      // FIXME: VARCHAR is defined in drill as utf8 string.
+      // Is Drill expecting something other than a Java string in this setObject call?
+      // Should we be mapping Daffodil strings to Drill VAR16CHAR type?
+      //
+      String s = ise.getString();
+      cw.setObject(s);
+      break;
+    }
+//        .put("TIME", TypeProtos.MinorType.TIME) // requires conversion

Review Comment:
   Does Daffodil support date, time, timestamp and interval types?  If so we should include those as well. 



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

To unsubscribe, e-mail: dev-unsubscribe@drill.apache.org

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


Re: [PR] DRILL-8474: Add Daffodil Format Plugin (drill)

Posted by "mbeckerle (via GitHub)" <gi...@apache.org>.
mbeckerle commented on PR #2836:
URL: https://github.com/apache/drill/pull/2836#issuecomment-1906827568

   Ok, so the geo-ip UDF stuff has no special mechanisms or description about those resource files, so the generic code that "scans" must find them and drag them along automatically. 
   
   That's the behavior I want. 
   
   What is "Drill's 3rd Party Jar folder"? 
   
   If a magic folder just gets dragged over to all nodes, and drill uses a class loader that arranges for jars in that folder to be searched, then there is very little to do, since a DFDL schema can be just a set of jar files containing related resources, and the classes for Daffodil's own UDFs and layers which are java code extensions of its own kind. 


-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] WIP: Preliminary Review on adding Daffodil to Drill (drill)

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


##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilBatchReader.java:
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.drill.exec.store.daffodil;
+
+import java.io.InputStream;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+import org.apache.daffodil.japi.DataProcessor;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.physical.impl.scan.v3.ManagedReader;
+import org.apache.drill.exec.physical.impl.scan.v3.file.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.store.daffodil.schema.DaffodilDataProcessorFactory;
+import org.apache.drill.exec.store.dfs.easy.EasySubScan;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.drill.exec.store.daffodil.schema.DrillDaffodilSchemaUtils.daffodilDataProcessorToDrillSchema;
+
+
+public class DaffodilBatchReader implements ManagedReader {
+
+  private static final Logger logger = LoggerFactory.getLogger(DaffodilBatchReader.class);
+  private final DaffodilFormatConfig formatConfig;
+  private final RowSetLoader rowSetLoader;
+  private final CustomErrorContext errorContext;
+  private final DaffodilMessageParser dafParser;
+  private final boolean validationMode;
+
+  private final InputStream dataInputStream;
+
+  static class DaffodilReaderConfig {
+    final DaffodilFormatPlugin plugin;
+    DaffodilReaderConfig(DaffodilFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public DaffodilBatchReader (DaffodilReaderConfig readerConfig, EasySubScan scan, FileSchemaNegotiator negotiator) {
+
+    errorContext = negotiator.parentErrorContext();
+    this.formatConfig = readerConfig.plugin.getConfig();
+
+    this.validationMode = formatConfig.getValidationMode();
+
+    //
+    // FIXME: Next, a MIRACLE occurs.
+    //
+    // We get the dfdlSchemaURI filled in from the query, or a default config location
+    // We get the rootName (or null if not supplied) from the query, or a default config location
+    // We get the rootNamespace (or null if not supplied) from the query, or a default config location
+    // We get the validationMode (true/false) filled in from the query or a default config location
+    // We get the dataInputURI filled in from the query, or from a default config location
+    //
+    // For a first cut, let's just fake it. :-)
+
+    String rootName = null;
+    String rootNamespace = null;
+
+    URI dfdlSchemaURI;
+    URI dataInputURI;
+
+    try {
+      dfdlSchemaURI = new URI("schema/complexArray1.dfdl.xsd");
+      dataInputURI = new URI("data/complexArray1.dat");
+    } catch (URISyntaxException e) {
+      throw UserException.validationError(e)
+          .message("Error retrieving DFDL schema files")
+          .build(logger);
+    }
+
+
+    // given dfdlSchemaURI and validation settings, and rootName, rootNamespace optionally
+    // get the Daffodil DataProcessor (aka parser static information) that we need, and from that
+    // we get the DaffodilMesageParser, which is a stateful driver for daffodil that actually does
+    // parsing.
+    DaffodilDataProcessorFactory dpf = new DaffodilDataProcessorFactory();
+    DataProcessor dp;
+    try {
+      dp = dpf.getDataProcessor(dfdlSchemaURI, true, rootName, rootNamespace);
+    } catch (Exception e) {
+      throw UserException.dataReadError(e)
+          .message(String.format("Failed to get Daffodil DFDL processor for: %s", dfdlSchemaURI.toString()))
+          .addContext(errorContext).addContext(e.getMessage()).build(logger);
+    }
+    // Create the corresponding Drill schema
+    TupleMetadata drillSchema = daffodilDataProcessorToDrillSchema(dp);
+    // Inform Drill about the schema
+    negotiator.tableSchema(drillSchema, true);
+    //
+    // FIXME: Now a MIRACLE occurs. We get the drill row writer (actually a rowSetLoader)??
+    //
+    rowSetLoader = negotiator.build().writer(); // FIXME: is this right?
+
+    // We construct the Daffodil InfosetOutputter which the daffodil parser uses to
+    // convert infoset event calls to fill in a Drill row via a rowSetLoader.
+    DaffodilDrillInfosetOutputter outputter = new DaffodilDrillInfosetOutputter(rowSetLoader);
+    // Now we can setup the dafParser with the outputter it will drive with the parser-produced
+    // infoset.
+    dafParser = new DaffodilMessageParser(dp); // needs further initialization after this.
+    dafParser.setInfosetOutputter(outputter);
+    // Lastly, we open the data stream
+    try {
+      dataInputStream = dataInputURI.toURL().openStream();

Review Comment:
   For the data files this works. 
   
   For schemas, this will not be a solution even temporarily. Daffodil loads schemas from the classpath. Large schemas are complex objects, akin to a software system with dependencies expressed via XML Schema include/import statements with schemaLocation attributes that contain relative URLs or "absolute" URLs where absolute means relative to some root of some jar file on the classpath. 
   
   Even simple DFDL schemas are routinely spread over a couple jars. 
   
   



-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] DRILL-8474: Add Daffodil Format Plugin (drill)

Posted by "cgivre (via GitHub)" <gi...@apache.org>.
cgivre commented on code in PR #2836:
URL: https://github.com/apache/drill/pull/2836#discussion_r1451756527


##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/schema/DaffodilDataProcessorFactory.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.drill.exec.store.daffodil.schema;
+
+import org.apache.daffodil.japi.Compiler;
+import org.apache.daffodil.japi.Daffodil;
+import org.apache.daffodil.japi.DataProcessor;
+import org.apache.daffodil.japi.Diagnostic;
+import org.apache.daffodil.japi.InvalidParserException;
+import org.apache.daffodil.japi.InvalidUsageException;
+import org.apache.daffodil.japi.ProcessorFactory;
+import org.apache.daffodil.japi.ValidationMode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.nio.channels.Channels;
+import java.util.List;
+import java.util.Objects;
+
+/**
+ * Compiles a DFDL schema (mostly for tests) or loads a pre-compiled DFDL schema so that one can
+ * obtain a DataProcessor for use with DaffodilMessageParser.
+ * <p/>
+ * TODO: Needs to use a cache to avoid reloading/recompiling every time.
+ */
+public class DaffodilDataProcessorFactory {
+  // Default constructor is used.
+
+  private static final Logger logger = LoggerFactory.getLogger(DaffodilDataProcessorFactory.class);
+
+  private DataProcessor dp;
+
+  /**
+   * Gets a Daffodil DataProcessor given the necessary arguments to compile or reload it.
+   *
+   * @param schemaFileURI
+   *     pre-compiled dfdl schema (.bin extension) or DFDL schema source (.xsd extension)
+   * @param validationMode
+   *     Use true to request Daffodil built-in 'limited' validation. Use false for no validation.
+   * @param rootName
+   *     Local name of root element of the message. Can be null to use the first element declaration
+   *     of the primary schema file. Ignored if reloading a pre-compiled schema.
+   * @param rootNS
+   *     Namespace URI as a string. Can be null to use the target namespace of the primary schema
+   *     file or if it is unambiguous what element is the rootName. Ignored if reloading a
+   *     pre-compiled schema.
+   * @return the DataProcessor
+   * @throws CompileFailure
+   *     - if schema compilation fails
+   */
+  public DataProcessor getDataProcessor(URI schemaFileURI, boolean validationMode, String rootName,
+      String rootNS)
+      throws CompileFailure {
+
+    DaffodilDataProcessorFactory dmp = new DaffodilDataProcessorFactory();
+    boolean isPrecompiled = schemaFileURI.toString().endsWith(".bin");
+    if (isPrecompiled) {
+      if (Objects.nonNull(rootName) && !rootName.isEmpty()) {
+        // A usage error. You shouldn't supply the name and optionally namespace if loading
+        // precompiled schema because those are built into it. Should be null or "".
+        logger.warn("Root element name '{}' is ignored when used with precompiled DFDL schema.",
+            rootName);
+      }
+      try {
+        dmp.loadSchema(schemaFileURI);
+      } catch (IOException | InvalidParserException e) {
+        throw new CompileFailure(e);

Review Comment:
   My thought here would be to fail as quickly as possible.  If the DFDL schema can't be read, I'm assuming that we cannot proceed, so throwing an exception would be the right thing to do IMHO.  With that said, we should make sure we provide a good error message that would explain what went wrong. 
   One of the issues we worked on for a while with Drill was that it would fail and you'd get a stack trace w/o a clear idea of what the actual issue is and how to rectify 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: dev-unsubscribe@drill.apache.org

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


Re: [PR] DRILL-8474: Add Daffodil Format Plugin (drill)

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


##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilDrillInfosetOutputter.java:
##########
@@ -0,0 +1,296 @@
+/*
+ * 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.drill.exec.store.daffodil;
+
+import org.apache.daffodil.runtime1.api.ComplexElementMetadata;
+import org.apache.daffodil.runtime1.api.ElementMetadata;
+import org.apache.daffodil.runtime1.api.InfosetArray;
+import org.apache.daffodil.runtime1.api.InfosetComplexElement;
+import org.apache.daffodil.japi.infoset.InfosetOutputter;
+import org.apache.daffodil.runtime1.api.InfosetSimpleElement;
+import org.apache.daffodil.runtime1.api.PrimitiveType;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.store.daffodil.schema.DrillDaffodilSchemaUtils;
+import org.apache.drill.exec.store.daffodil.schema.DrillDaffodilSchemaVisitor;
+import org.apache.drill.exec.vector.accessor.ArrayWriter;
+import org.apache.drill.exec.vector.accessor.ColumnWriter;
+import org.apache.drill.exec.vector.accessor.ObjectType;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Stack;
+
+/**
+ * Adapts Daffodil parser infoset event calls to Drill writer calls
+ * to fill in Drill data rows.
+ */
+public class DaffodilDrillInfosetOutputter
+    extends InfosetOutputter {
+
+  private boolean isOriginalRoot() {
+    boolean result = currentTupleWriter() == rowSetWriter;

Review Comment:
   Next commit will have files reformatted based on the eclipse settings in the dev-support/formatter directory, as implemented by intelliJ IDEA when those settings were imported. 



-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] DRILL-8474: Add Daffodil Format Plugin (drill)

Posted by "mbeckerle (via GitHub)" <gi...@apache.org>.
mbeckerle commented on PR #2836:
URL: https://github.com/apache/drill/pull/2836#issuecomment-1883962208

   > @mbeckerle With respect to style, I tried to reply to that comment, but the thread won't let me. In any event, Drill classes will typically start with the constructor, then have whatever methods are appropriate for the class. The logger creation usually happens before the constructor. I think all of your other classes followed this format, so the one or two that didn't kind of jumped out at me.
   
   @cgivre I believe the style issues are all fixed. The build did not get any codestyle issues. 


-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] DRILL-8474: Add Daffodil Format Plugin (drill)

Posted by "cgivre (via GitHub)" <gi...@apache.org>.
cgivre commented on PR #2836:
URL: https://github.com/apache/drill/pull/2836#issuecomment-1906689793

   > > > @cgivre @paul-rogers is there an example of a Drill UDF that is not part of the drill repository tree?
   > > > I'd like to understand the mechanisms for distributing any jar files and dependencies of the UDF that drill uses. I can't find any such in the quasi-USFs that are in the Drill tree, because well, since they are part of Drill, and so are their dependencies, this problem doesn't exist.
   > > 
   > > 
   > > @mbeckerle Here's an example: https://github.com/datadistillr/drill-humanname-functions. I'm sorry we weren't able to connect last week.
   > 
   > If I understand this correctly, if a jar is on the classpath and has drill-module.conf in its root dir, then drill will find it and read that HOCON file to get the package to add to drill.classpath.scanning.packages.
   
   I believe that is correct.
   
   > 
   > Drill then appears to scan jars for class files for those packages. Not sure what it is doing with the class files. I imagine it is repackaging them somehow so Drill can use them on the drill distributed nodes. But it isn't yet clear to me how this aspect works. Do these classes just get loaded on the distributed drill nodes? Or is the classpath augmented in some way on the drill nodes so that they see a jar that contains all these classes?
   > 
   > I have two questions:
   > 
   > (1) what about dependencies? The UDF may depend on libraries which depend on other libraries, etc.
   
   So UDFs are a bit of a special case, but if they do have dependencies, you have to also include those JAR files in the UDF directory, or in Drill's 3rd party JAR folder.   I'm not that good with maven, but I've often wondered about making a so-called fat-JAR which includes the dependencies as part of the UDF JAR file.
   
   > 
   > (2) what about non-class files, e.g., things under src/main/resources of the project that go into the jar, but aren't "class" files? How do those things also get moved? How would code running in the drill node access these? The usual method is to call getResource(URL) with a URL that gives the path within a jar file to the resource in question.
   
   Take a look at this UDF. https://github.com/datadistillr/drill-geoip-functions
   This UDF has a few external resources including a CSV file and the MaxMind databases.
   
   
   > 
   > Thanks for any info.
   
   


-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] DRILL-8474: Add Daffodil Format Plugin (drill)

Posted by "mbeckerle (via GitHub)" <gi...@apache.org>.
mbeckerle commented on PR #2836:
URL: https://github.com/apache/drill/pull/2836#issuecomment-2081176156

   Creating a new squashed PR so as to avoid loss of the comments on this PR. 


-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] DRILL-8474: Add Daffodil Format Plugin (drill)

Posted by "mbeckerle (via GitHub)" <gi...@apache.org>.
mbeckerle closed pull request #2836: DRILL-8474: Add Daffodil Format Plugin
URL: https://github.com/apache/drill/pull/2836


-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] WIP: Preliminary Review on adding Daffodil to Drill (drill)

Posted by "cgivre (via GitHub)" <gi...@apache.org>.
cgivre commented on code in PR #2836:
URL: https://github.com/apache/drill/pull/2836#discussion_r1375364472


##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilDrillInfosetOutputter.java:
##########
@@ -1,25 +1,21 @@
 package org.apache.drill.exec.store.daffodil;
 
-import org.apache.daffodil.japi.infoset.InfosetOutputter;
 import org.apache.daffodil.lib.xml.NamedQName;
-import org.apache.daffodil.runtime1.dpath.NodeInfo;
-import org.apache.daffodil.runtime1.infoset.DIArray;
-import org.apache.daffodil.runtime1.infoset.DIComplex;
-import org.apache.daffodil.runtime1.infoset.DISimple;
-import org.apache.daffodil.runtime1.processors.ElementRuntimeData;
+import org.apache.daffodil.runtime1.api.*;

Review Comment:
   Nit:  We don't allow star imports in Drill.... Sorry.



-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] WIP: Preliminary Review on adding Daffodil to Drill (drill)

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


##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/schema/DrillDaffodilSchemaVisitor.java:
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.drill.exec.store.daffodil.schema;
+
+import org.apache.daffodil.runtime1.api.ComplexElementMetadata;
+import org.apache.daffodil.runtime1.api.MetadataHandler;
+import org.apache.daffodil.runtime1.api.SimpleElementMetadata;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.record.metadata.MapBuilder;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Stack;
+
+/**
+ * This class transforms a DFDL/Daffodil schema into a Drill Schema.
+ */
+public class DrillDaffodilSchemaVisitor
+     extends MetadataHandler
+{
+  private static final Logger logger = LoggerFactory.getLogger(DrillDaffodilSchemaVisitor.class);
+  private final SchemaBuilder builder = new SchemaBuilder();
+
+  private final Stack<MapBuilder> mapBuilderStack = new Stack<MapBuilder>();
+
+  private MapBuilder mapBuilder() {
+    return mapBuilderStack.peek();
+  }
+
+  /**
+   * Returns a {@link TupleMetadata} representation of the DFDL schema.
+   * Should only be called after the walk of the DFDL schema with this visitor has been called.
+   * @return A {@link TupleMetadata} representation of the DFDL schema.
+   */
+  public TupleMetadata getDrillSchema() {
+    return builder.build();
+  }
+
+  @Override
+  public void elementSimple(SimpleElementMetadata md) {

Review Comment:
   Daffodil should rename this to simpleMetadata, and below to complexMetadata. They sound too much like data right now. 



##########
exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FormatCreator.java:
##########
@@ -117,7 +117,7 @@ private static Map<Class<?>, Constructor<?>> initConfigConstructors(Collection<C
         } catch (InstantiationException | IllegalAccessException | IllegalArgumentException | InvocationTargetException e1) {
           logger.warn("Failure initializing storage config named '{}' of type '{}'.", e.getKey(), e.getValue().getClass().getName(), e1);
         }
-      }
+      }xml

Review Comment:
   ??? Looks like I clobbered this by accident. Revert



##########
exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FormatCreator.java:
##########
@@ -117,7 +117,7 @@ private static Map<Class<?>, Constructor<?>> initConfigConstructors(Collection<C
         } catch (InstantiationException | IllegalAccessException | IllegalArgumentException | InvocationTargetException e1) {
           logger.warn("Failure initializing storage config named '{}' of type '{}'.", e.getKey(), e.getValue().getClass().getName(), e1);
         }
-      }
+      }xml

Review Comment:
   Fixed already. 



##########
contrib/format-daffodil/src/test/java/org/apache/drill/exec/store/daffodil/TestDaffodilUtils.java:
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.drill.exec.store.daffodil;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestDaffodilUtils {
+
+//  @Test

Review Comment:
   Not sure I'm going to need this file. Perhaps delete. 



##########
contrib/format-daffodil/src/test/java/org/apache/drill/exec/store/daffodil/TestDaffodilReader.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.drill.exec.store.daffodil;
+
+import org.apache.drill.categories.RowSetTest;
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.rowSet.RowSet;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.test.ClusterFixture;
+import org.apache.drill.test.ClusterTest;
+import org.apache.drill.test.QueryBuilder;
+import org.apache.drill.test.rowSet.RowSetComparison;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.nio.file.Paths;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+
+import static org.apache.drill.test.QueryTestUtil.generateCompressedFile;
+import static org.apache.drill.test.rowSet.RowSetUtilities.mapArray;
+import static org.apache.drill.test.rowSet.RowSetUtilities.objArray;
+import static org.apache.drill.test.rowSet.RowSetUtilities.strArray;
+import static org.junit.Assert.assertEquals;
+
+@Category(RowSetTest.class)
+public class TestDaffodilReader extends ClusterTest {
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    // boilerplate call to start test rig
+    ClusterTest.startCluster(ClusterFixture.builder(dirTestWatcher));
+
+    //
+    // FIXME: isn't this supposed to come from some file location?
+    //
+    // FIXME: Where does drill search for these configs?
+    // FIXME: What kind of file, what naming convention and what format?
+    DaffodilFormatConfig formatConfig =
+        new DaffodilFormatConfig(null,
+            "",
+            "",
+            "",
+            false,
+            "");
+
+    cluster.defineFormat("dfs", "daffodil", formatConfig);
+
+    // Needed to test against compressed files.
+    // Copies data from src/test/resources to the dfs root.
+    java.io.File newDataDir = dirTestWatcher.copyResourceToRoot(Paths.get("data/"));
+    java.io.File newSchemaDir = dirTestWatcher.copyResourceToRoot(Paths.get("schema/"));
+  }
+
+  /**
+   * This unit test tests a simple data file
+   *
+   * @throws Exception Throw exception if anything goes wrong
+   */
+  @Test
+  public void testSimple1() throws Exception {
+

Review Comment:
   @cgivre 
   I could use some help figuring out why I needed absolute URIs here. 
   
   I thought the `copyResourceToRoot` calls above in the setup would make those dirs available to the test, but paths like 'schema/simple.dfdl.xsd' did not work nor 'data/simple.dat.gz'. 
   
   Ultimately I want to drop dataURI and have the
   ```
   dfs.`data/simple.dat.gz`
   ```
   find the file, but I had no luck with that so just did this dataURI workaround. 



-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] DRILL-2835: Daffodil Feature for Drill (drill)

Posted by "cgivre (via GitHub)" <gi...@apache.org>.
cgivre commented on PR #2836:
URL: https://github.com/apache/drill/pull/2836#issuecomment-1867184695

   > > > Rebased onto latest Drill master as of 2023-12-21 (force pushed one more time)
   > > > Note that this is never going to pass automated tests until the Daffodil release this depends on is official (currently it needs a locally build Daffodil 3.7.0-snapshot, though the main daffodil branch has the changes integrated so any 3.7.0-snapshot build will work.
   > > 
   > > 
   > > @mbeckerle This is really great work! Thanks for your persistence on this. Do you have a an ETA on the next Daffodil release?
   > 
   > We could have a Daffodil release in Jan or Feb. There are some Daffodil API cleanups that need to be discussed that would provide better stability for this Drill integration ... we may want to wait for those and update this to use them.
   
   @mbeckerle So is the next step really to figure out how to access the Daffodil files from a potentially distributed environment?


-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] DRILL-8474: Add Daffodil Format Plugin (drill)

Posted by "mbeckerle (via GitHub)" <gi...@apache.org>.
mbeckerle commented on PR #2836:
URL: https://github.com/apache/drill/pull/2836#issuecomment-1902750285

   @cgivre @paul-rogers is there an example of a Drill UDF that is not part of the drill repository tree? 
   
   I'd like to understand the mechanisms for distributing any jar files and dependencies of the UDF that drill uses. I can't find any such in the quasi-USFs that are in the Drill tree, because well, since they are part of Drill, and so are their dependencies, this problem doesn't exist. 


-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] DRILL-8474: Add Daffodil Format Plugin (drill)

Posted by "mbeckerle (via GitHub)" <gi...@apache.org>.
mbeckerle commented on PR #2836:
URL: https://github.com/apache/drill/pull/2836#issuecomment-1877915796

   This is ready for a next review. All the scalar types are now implemented with typed setter calls. 
   
   The prior review comments have all been addressed I believe.
   
   Remaining things to do include:
   
   1. How to get the compiled DFDL schema object so it can be loaded by daffodil out at the distributed Drill nodes.
   2. Test of nilled values (and more tests generally to show deeply nested and repeating nested objects work.)
   3. Errors - revisit every place errors are detected or thrown to make sure these are being done the right way for DFDL schema compilation and runtime errors as well. 
   


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

To unsubscribe, e-mail: dev-unsubscribe@drill.apache.org

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


Re: [PR] DRILL-8474: Add Daffodil Format Plugin (drill)

Posted by "cgivre (via GitHub)" <gi...@apache.org>.
cgivre commented on code in PR #2836:
URL: https://github.com/apache/drill/pull/2836#discussion_r1451758017


##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilBatchReader.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.drill.exec.store.daffodil;
+
+import org.apache.daffodil.japi.DataProcessor;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.physical.impl.scan.v3.ManagedReader;
+import org.apache.drill.exec.physical.impl.scan.v3.file.FileDescrip;
+import org.apache.drill.exec.physical.impl.scan.v3.file.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.store.daffodil.schema.DaffodilDataProcessorFactory;
+import org.apache.drill.exec.store.dfs.DrillFileSystem;
+import org.apache.drill.exec.store.dfs.easy.EasySubScan;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Objects;
+
+import static org.apache.drill.exec.store.daffodil.schema.DaffodilDataProcessorFactory.*;
+import static org.apache.drill.exec.store.daffodil.schema.DrillDaffodilSchemaUtils.daffodilDataProcessorToDrillSchema;
+
+public class DaffodilBatchReader implements ManagedReader {
+
+  private static final Logger logger = LoggerFactory.getLogger(DaffodilBatchReader.class);
+  private final RowSetLoader rowSetLoader;
+  private final CustomErrorContext errorContext;
+  private final DaffodilMessageParser dafParser;
+  private final InputStream dataInputStream;
+
+  public DaffodilBatchReader(DaffodilReaderConfig readerConfig, EasySubScan scan,
+      FileSchemaNegotiator negotiator) {
+
+    errorContext = negotiator.parentErrorContext();
+    DaffodilFormatConfig dafConfig = readerConfig.plugin.getConfig();
+
+    String schemaURIString = dafConfig.getSchemaURI(); // "schema/complexArray1.dfdl.xsd";
+    String rootName = dafConfig.getRootName();
+    String rootNamespace = dafConfig.getRootNamespace();
+    boolean validationMode = dafConfig.getValidationMode();
+
+    URI dfdlSchemaURI;
+    try {
+      dfdlSchemaURI = new URI(schemaURIString);
+    } catch (URISyntaxException e) {
+      throw UserException.validationError(e).build(logger);
+    }
+
+    FileDescrip file = negotiator.file();
+    DrillFileSystem fs = file.fileSystem();
+    URI fsSchemaURI = fs.getUri().resolve(dfdlSchemaURI);
+
+    DaffodilDataProcessorFactory dpf = new DaffodilDataProcessorFactory();
+    DataProcessor dp;
+    try {
+      dp = dpf.getDataProcessor(fsSchemaURI, validationMode, rootName, rootNamespace);
+    } catch (CompileFailure e) {
+      throw UserException.dataReadError(e)
+          .message(String.format("Failed to get Daffodil DFDL processor for: %s", fsSchemaURI))
+          .addContext(errorContext).addContext(e.getMessage()).build(logger);
+    }
+    // Create the corresponding Drill schema.
+    // Note: this could be a very large schema. Think of a large complex RDBMS schema,
+    // all of it, hundreds of tables, but all part of the same metadata tree.
+    TupleMetadata drillSchema = daffodilDataProcessorToDrillSchema(dp);
+    // Inform Drill about the schema
+    negotiator.tableSchema(drillSchema, true);
+
+    //
+    // DATA TIME: Next we construct the runtime objects, and open files.
+    //
+    // We get the DaffodilMessageParser, which is a stateful driver for daffodil that
+    // actually does the parsing.
+    rowSetLoader = negotiator.build().writer();
+
+    // We construct the Daffodil InfosetOutputter which the daffodil parser uses to
+    // convert infoset event calls to fill in a Drill row via a rowSetLoader.
+    DaffodilDrillInfosetOutputter outputter = new DaffodilDrillInfosetOutputter(rowSetLoader);
+
+    // Now we can set up the dafParser with the outputter it will drive with
+    // the parser-produced infoset.
+    dafParser = new DaffodilMessageParser(dp); // needs further initialization after this.
+    dafParser.setInfosetOutputter(outputter);
+
+    Path dataPath = file.split().getPath();
+    // Lastly, we open the data stream
+    try {
+      dataInputStream = fs.openPossiblyCompressedStream(dataPath);
+    } catch (IOException e) {
+      throw UserException.dataReadError(e)
+          .message(String.format("Failed to open input file: %s", dataPath.toString()))
+          .addContext(errorContext).addContext(e.getMessage()).build(logger);
+    }
+    // And lastly,... tell daffodil the input data stream.
+    dafParser.setInputStream(dataInputStream);
+  }
+
+  /**
+   * This is the core of actual processing - data movement from Daffodil to Drill.
+   * <p>
+   * If there is space in the batch, and there is data available to parse then this calls the
+   * daffodil parser, which parses data, delivering it to the rowWriter by way of the infoset
+   * outputter.
+   * <p>
+   * Repeats until the rowWriter is full (a batch is full), or there is no more data, or a parse
+   * error ends execution with a throw.
+   * <p>
+   * Validation errors and other warnings are not errors and are logged but do not cause parsing to
+   * fail/throw.
+   *
+   * @return true if there are rows retrieved, false if no rows were retrieved, which means no more
+   *     will ever be retrieved (end of data).
+   * @throws RuntimeException
+   *     on parse errors.
+   */
+  @Override
+  public boolean next() {
+    // Check assumed invariants
+    // We don't know if there is data or not. This could be called on an empty data file.
+    // We DO know that this won't be called if there is no space in the batch for even 1
+    // row.
+    if (dafParser.isEOF()) {
+      return false; // return without even checking for more rows or trying to parse.
+    }
+    while (rowSetLoader.start() && !dafParser.isEOF()) { // we never zero-trip this loop.
+      // the predicate is always true once.
+      dafParser.parse();
+      if (dafParser.isProcessingError()) {
+        assert (Objects.nonNull(dafParser.getDiagnostics()));
+        throw UserException.dataReadError().message(dafParser.getDiagnosticsAsString())
+            .addContext(errorContext).build(logger);
+      }
+      if (dafParser.isValidationError()) {
+        logger.warn(dafParser.getDiagnosticsAsString());

Review Comment:
   @mbeckerle The question I'd have is whether the query can proceed if validation fails.  (I don't know the answer)
   If the answer is no, then we need to halt execution ASAP and throw an exception.   If the answer is it can proceed, but the data might be less than ideal, maybe we add a configuration option which will allow the user to decide the behavior on a validation failure.  
   
   I could imagine situations where you have Drill unable to read a huge file because someone fat fingered a quotation mark somewhere or something like that.  In a situation like that, sometimes you might just want to say I'll accept a row or two of bad data just so I can read the whole file. 
   
   



-- 
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: dev-unsubscribe@drill.apache.org

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


Re: [PR] DRILL-8474: Add Daffodil Format Plugin (drill)

Posted by "cgivre (via GitHub)" <gi...@apache.org>.
cgivre commented on code in PR #2836:
URL: https://github.com/apache/drill/pull/2836#discussion_r1451756763


##########
contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/schema/DaffodilDataProcessorFactory.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.drill.exec.store.daffodil.schema;
+
+import org.apache.daffodil.japi.Compiler;
+import org.apache.daffodil.japi.Daffodil;
+import org.apache.daffodil.japi.DataProcessor;
+import org.apache.daffodil.japi.Diagnostic;
+import org.apache.daffodil.japi.InvalidParserException;
+import org.apache.daffodil.japi.InvalidUsageException;
+import org.apache.daffodil.japi.ProcessorFactory;
+import org.apache.daffodil.japi.ValidationMode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.nio.channels.Channels;
+import java.util.List;
+import java.util.Objects;
+
+/**
+ * Compiles a DFDL schema (mostly for tests) or loads a pre-compiled DFDL schema so that one can
+ * obtain a DataProcessor for use with DaffodilMessageParser.
+ * <p/>
+ * TODO: Needs to use a cache to avoid reloading/recompiling every time.
+ */
+public class DaffodilDataProcessorFactory {
+  // Default constructor is used.
+
+  private static final Logger logger = LoggerFactory.getLogger(DaffodilDataProcessorFactory.class);
+
+  private DataProcessor dp;
+
+  /**
+   * Gets a Daffodil DataProcessor given the necessary arguments to compile or reload it.
+   *
+   * @param schemaFileURI
+   *     pre-compiled dfdl schema (.bin extension) or DFDL schema source (.xsd extension)
+   * @param validationMode
+   *     Use true to request Daffodil built-in 'limited' validation. Use false for no validation.
+   * @param rootName
+   *     Local name of root element of the message. Can be null to use the first element declaration
+   *     of the primary schema file. Ignored if reloading a pre-compiled schema.
+   * @param rootNS
+   *     Namespace URI as a string. Can be null to use the target namespace of the primary schema
+   *     file or if it is unambiguous what element is the rootName. Ignored if reloading a
+   *     pre-compiled schema.
+   * @return the DataProcessor
+   * @throws CompileFailure
+   *     - if schema compilation fails
+   */
+  public DataProcessor getDataProcessor(URI schemaFileURI, boolean validationMode, String rootName,
+      String rootNS)
+      throws CompileFailure {
+
+    DaffodilDataProcessorFactory dmp = new DaffodilDataProcessorFactory();
+    boolean isPrecompiled = schemaFileURI.toString().endsWith(".bin");
+    if (isPrecompiled) {
+      if (Objects.nonNull(rootName) && !rootName.isEmpty()) {
+        // A usage error. You shouldn't supply the name and optionally namespace if loading
+        // precompiled schema because those are built into it. Should be null or "".
+        logger.warn("Root element name '{}' is ignored when used with precompiled DFDL schema.",
+            rootName);
+      }
+      try {
+        dmp.loadSchema(schemaFileURI);
+      } catch (IOException | InvalidParserException e) {
+        throw new CompileFailure(e);
+      }
+      dmp.setupDP(validationMode, null);
+    } else {
+      List<Diagnostic> pfDiags;
+      try {
+        pfDiags = dmp.compileSchema(schemaFileURI, rootName, rootNS);
+      } catch (URISyntaxException | IOException e) {
+        throw new CompileFailure(e);
+      }
+      dmp.setupDP(validationMode, pfDiags);
+    }
+    return dmp.dp;
+  }
+
+  private void loadSchema(URI schemaFileURI) throws IOException, InvalidParserException {
+    Compiler c = Daffodil.compiler();
+    dp = c.reload(Channels.newChannel(schemaFileURI.toURL().openStream()));

Review Comment:
   This definitely seems like an area where there is potential for a lot of different things to go wrong.  My view is we should just do our best to provide clear error messages so that the user can identify and fix the issues.



-- 
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: dev-unsubscribe@drill.apache.org

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