diff --git a/contrib/format-daffodil/README.md b/contrib/format-daffodil/README.md
new file mode 100644
index 00000000000..6418f94d10b
--- /dev/null
+++ b/contrib/format-daffodil/README.md
@@ -0,0 +1,37 @@
+# Daffodil Format Reader
+This plugin enables Drill to read DFDL-described data from files by way of the Apache Daffodil DFDL implementation.
+
+## Configuration:
+To use Daffodil schemata, simply add the following to the `formats` section of a file-based storage plugin:
+
+```json
+"daffodil": {
+ "type": "daffodil",
+ "extensions": [
+ "dat"
+ ]
+ }
+```
+There are four other optional parameters which you can specify:
+* `schemaURI`: Pre-compiled dfdl schema (.bin extension) or DFDL schema source (.xsd extension)
+* `validationMode`: Use `true` to request Daffodil built-in limited validation. Use `false` for no validation.
+* `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.
+* `rootNameSpace`: 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.
+
+## Usage:
+
+
+
+## Limitations:
+At the moment, the DFDL schema is found on the local file system, which won't support Drill's distributed architecture.
+
+There are restrictions on the DFDL schemas that this can handle. In particular, all element children must have distinct element names, including across choice branches. Unfortunately, this rules out a number of large DFDL schemas.
+
+TBD: Auto renaming as part of the Daffodil-to-Drill metadata mapping?
+
+The data is parsed fully from its native form into a Drill data structure held in memory. No attempt is made to avoid access to parts of the DFDL-described data that are not needed to answer the query.
+
+If the data is not well-formed, an error occurs and the query fails.
+
+If the data is invalid, and validity checking by Daffodil is enabled, then an error occurs and the query fails.
+
diff --git a/contrib/format-daffodil/pom.xml b/contrib/format-daffodil/pom.xml
new file mode 100644
index 00000000000..eaa4784736e
--- /dev/null
+++ b/contrib/format-daffodil/pom.xml
@@ -0,0 +1,94 @@
+
+
+
+ 4.0.0
+
+
+ drill-contrib-parent
+ org.apache.drill.contrib
+ 1.23.0-SNAPSHOT
+
+
+ drill-format-daffodil
+ Drill : Contrib : Format : Daffodil
+
+
+
+ org.apache.drill.exec
+ drill-java-exec
+ ${project.version}
+
+
+ org.apache.daffodil
+ daffodil-japi_2.13
+ ${daffodil.version}
+
+
+ org.apache.daffodil
+ daffodil-runtime1_2.13
+ ${daffodil.version}
+
+
+
+ org.apache.drill.exec
+ drill-java-exec
+ tests
+ ${project.version}
+ test
+
+
+
+ org.apache.drill
+ drill-common
+ tests
+ ${project.version}
+ test
+
+
+
+
+
+
+ maven-resources-plugin
+
+
+ copy-java-sources
+ process-sources
+
+ copy-resources
+
+
+ ${basedir}/target/classes/org/apache/drill/exec/store/daffodil
+
+
+
+ src/main/java/org/apache/drill/exec/store/daffodil
+ true
+
+
+
+
+
+
+
+
+
diff --git a/contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilBatchReader.java b/contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilBatchReader.java
new file mode 100644
index 00000000000..d3d233ca03c
--- /dev/null
+++ b/contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilBatchReader.java
@@ -0,0 +1,197 @@
+/*
+ * 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.CompileFailure;
+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 schemaFile = dafConfig.getSchemaFile();
+ String schemaURIString = dafConfig.getSchemaURI();
+ String rootName = dafConfig.getRootName();
+ String rootNamespace = dafConfig.getRootNamespace();
+ boolean validationMode = dafConfig.getValidationMode();
+
+ // Determine the schema URI:
+ // - If schemaFile is provided, it takes precedence and is looked up in the registry area
+ // - Otherwise, use schemaURI (full path)
+ URI dfdlSchemaURI;
+ try {
+ if (schemaFile != null && !schemaFile.isEmpty()) {
+ // schemaFile takes precedence - construct path from registry area
+ Path registryArea = readerConfig.plugin.getContext()
+ .getRemoteDaffodilSchemaRegistry().getRegistryArea();
+ Path schemaPath = new Path(registryArea, schemaFile);
+ dfdlSchemaURI = schemaPath.toUri();
+ } else if (schemaURIString != null && !schemaURIString.isEmpty()) {
+ // Use the provided schemaURI
+ dfdlSchemaURI = new URI(schemaURIString);
+ } else {
+ // Neither provided - will result in empty URI
+ dfdlSchemaURI = new URI("");
+ }
+ } 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.
+ *
+ * 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.
+ *
+ * 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.
+ *
+ * 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());
+ // Note that even if daffodil is set to not validate, validation errors may still occur
+ // from DFDL's "recoverableError" assertions.
+ }
+ 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);
+ }
+}
+
+class DaffodilReaderConfig {
+ final DaffodilFormatPlugin plugin;
+
+ DaffodilReaderConfig(DaffodilFormatPlugin plugin) {
+ this.plugin = plugin;
+ }
+}
diff --git a/contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilDrillInfosetOutputter.java b/contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilDrillInfosetOutputter.java
new file mode 100644
index 00000000000..189d2e0c18b
--- /dev/null
+++ b/contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilDrillInfosetOutputter.java
@@ -0,0 +1,393 @@
+/*
+ * 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 com.ibm.icu.util.Calendar;
+import com.ibm.icu.util.TimeZone;
+import org.apache.daffodil.japi.infoset.InfosetOutputter;
+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.InfosetSimpleElement;
+import org.apache.daffodil.runtime1.api.DFDLPrimType;
+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.ObjectType;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigDecimal;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.ZoneId;
+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 static final Logger logger = LoggerFactory.getLogger(DaffodilDrillInfosetOutputter.class);
+ /**
+ * Stack that is used only if we have sub-structures that are not simple-type fields of the row.
+ */
+ private final Stack tupleWriterStack = new Stack<>();
+ private final Stack arrayWriterStack = new Stack<>();
+ /**
+ * 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;
+ private RowSetLoader rowSetWriter;
+
+ private DaffodilDrillInfosetOutputter() {
+ } // no default constructor
+
+ public DaffodilDrillInfosetOutputter(RowSetLoader writer) {
+ this.rowSetWriter = writer;
+ this.tupleWriterStack.push(writer);
+ }
+
+ private static void nyi() {
+ throw new IllegalStateException("not yet implemented.");
+ }
+
+ private static void fatalError(String s) {
+ throw new IllegalStateException(s);
+ }
+
+ private boolean isOriginalRoot() {
+ boolean result = currentTupleWriter() == rowSetWriter;
+ if (result) {
+ assert (tupleWriterStack.size() == 1);
+ }
+ return result;
+ }
+
+ private TupleWriter currentTupleWriter() {
+ return tupleWriterStack.peek();
+ }
+
+ private ArrayWriter currentArrayWriter() {
+ return arrayWriterStack.peek();
+ }
+
+ @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);
+ ScalarWriter 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().scalar(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
+ // event calls for the children.
+ isRootElement = false;
+ return;
+ }
+ if (md.isArray()) {
+ assert (!arrayWriterStack.isEmpty());
+ 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 invariantFailed(String dafTypeName, ColumnMetadata cm) {
+ String msg = String.format(
+ "Daffodil to Drill Conversion Invariant Failed: dafType %s, drill type %s.", dafTypeName,
+ cm.typeString());
+ logger.error(msg);
+ fatalError(msg);
+ }
+
+ private void convertDaffodilValueToDrillValue(InfosetSimpleElement ise, ColumnMetadata cm,
+ ScalarWriter cw) {
+ DFDLPrimType dafType = ise.metadata().dfdlType();
+ String dafTypeName = dafType.name();
+ TypeProtos.MinorType drillType = DrillDaffodilSchemaUtils.getDrillDataType(dafType);
+ assert (drillType == cm.type());
+ switch (drillType) {
+ case BIGINT: { // BIGINT type is not a Java BigInteger, BIGINT is a signed 8-byte long in Drill.
+ switch (dafType) {
+ case UnsignedInt: {
+ cw.setLong(ise.getUnsignedInt());
+ break;
+ }
+ case Long: {
+ cw.setLong(ise.getLong());
+ break;
+ }
+ default:
+ invariantFailed(dafTypeName, cm);
+ }
+ break;
+ }
+ case INT: {
+ cw.setInt(ise.getInt());
+ break;
+ }
+ case SMALLINT: {
+ cw.setInt(ise.getShort()); // there is no setShort
+ break;
+ }
+ case TINYINT: {
+ cw.setInt(ise.getByte()); // there is no setByte
+ break;
+ }
+ case UINT4: {
+ // daffodil represents unsigned int as long.
+ // drill represents unsigned int as int.
+ cw.setInt(ise.getUnsignedInt().intValue());
+ break;
+ }
+ case UINT2: {
+ cw.setInt(ise.getUnsignedShort());
+ break;
+ }
+ case UINT1: {
+ cw.setInt(ise.getUnsignedByte());
+ break;
+ }
+ case VARDECIMAL: {
+ switch (dafType) {
+ case UnsignedLong: {
+ cw.setDecimal(new BigDecimal(ise.getUnsignedLong()));
+ break;
+ }
+ case Integer: {
+ cw.setDecimal(new BigDecimal(ise.getInteger()));
+ break;
+ }
+ case NonNegativeInteger: {
+ cw.setDecimal(new BigDecimal(ise.getNonNegativeInteger()));
+ break;
+ }
+ default:
+ invariantFailed(dafTypeName, cm);
+ }
+ break;
+ }
+ case BIT: {
+ cw.setBoolean(ise.getBoolean());
+ break;
+ }
+ case FLOAT8: {
+ switch (dafType) {
+ case Double: {
+ cw.setDouble(ise.getDouble());
+ break;
+ }
+ case Float: {
+ // converting a float to a double by doubleValue() fails here
+ // Float.MaxValue converted to a double via doubleValue()
+ // then placed in a FLOAT8 column displays as
+ // 3.4028234663852886E38 not 3.4028235E38.
+ // But converting to string first, then to double works properly.
+ cw.setDouble(Double.parseDouble(ise.getFloat().toString()));
+ break;
+ }
+ default:
+ invariantFailed(dafTypeName, cm);
+ }
+ break;
+ }
+ case FLOAT4: {
+ // we don't use float4, we always use float8.
+ invariantFailed(dafTypeName, cm);
+ // cw.setFloat(ise.getFloat());
+ break;
+ }
+ case VARBINARY: {
+ byte[] hexBinary = ise.getHexBinary();
+ cw.setBytes(hexBinary, hexBinary.length);
+ break;
+ }
+ case VARCHAR: {
+ switch (dafType) {
+ case Decimal: {
+ BigDecimal decimal = ise.getDecimal();
+ cw.setString(decimal.toString());
+ break;
+ }
+ case String: {
+ String s = ise.getString();
+ cw.setString(s);
+ break;
+ }
+ default:
+ invariantFailed(dafTypeName, cm);
+ }
+ break;
+ }
+ case TIME: {
+ Calendar icuCal = ise.getTime();
+ Instant instant = Instant.ofEpochMilli(icuCal.getTimeInMillis());
+ TimeZone icuZone = icuCal.getTimeZone();
+ String zoneString = icuZone.getID();
+ ZoneId zoneId = ZoneId.of(zoneString);
+ LocalTime localTime = instant.atZone(zoneId).toLocalTime();
+ cw.setTime(localTime);
+ break;
+ }
+ case DATE: {
+ Calendar icuCalendar = ise.getDate();
+ // Extract year, month, and day from ICU Calendar
+ int year = icuCalendar.get(Calendar.YEAR);
+ // Note: ICU Calendar months are zero-based, similar to java.util.Calendar
+ int month = icuCalendar.get(Calendar.MONTH) + 1;
+ int day = icuCalendar.get(Calendar.DAY_OF_MONTH);
+ // Create a LocalDate
+ LocalDate localDate = LocalDate.of(year, month, day);
+ cw.setDate(localDate);
+ break;
+ }
+ case TIMESTAMP: {
+ Calendar icuCalendar = ise.getDateTime();
+ // Get time in milliseconds from the epoch
+ long millis = icuCalendar.getTimeInMillis();
+ // Create an Instant from milliseconds
+ Instant instant = Instant.ofEpochMilli(millis);
+ cw.setTimestamp(instant);
+ break;
+ }
+ default:
+ invariantFailed(dafTypeName, cm);
+ }
+ }
+
+ private void DFDLParseError(String s) {
+ throw new RuntimeException(s);
+ }
+}
+
diff --git a/contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilFormatConfig.java b/contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilFormatConfig.java
new file mode 100644
index 00000000000..7b3f40e15d1
--- /dev/null
+++ b/contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilFormatConfig.java
@@ -0,0 +1,129 @@
+/*
+ * 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 com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.google.common.collect.ImmutableList;
+import org.apache.drill.common.PlanStringBuilder;
+import org.apache.drill.common.logical.FormatPluginConfig;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+@JsonTypeName(DaffodilFormatPlugin.DEFAULT_NAME)
+@JsonInclude(JsonInclude.Include.NON_DEFAULT)
+public class DaffodilFormatConfig implements FormatPluginConfig {
+
+ public final List extensions;
+ public final String schemaURI;
+ public final String schemaFile;
+ public final boolean validationMode;
+ public final String rootName;
+ public final String rootNamespace;
+
+ /**
+ * In the constructor for a format config, you should not use boxed versions of primitive types.
+ * It creates problems with defaulting them (they default to null) which cannot be unboxed.
+ */
+ @JsonCreator
+ public DaffodilFormatConfig(
+ @JsonProperty("extensions") List extensions,
+ @JsonProperty("schemaURI") String schemaURI,
+ @JsonProperty("schemaFile") String schemaFile,
+ @JsonProperty("rootName") String rootName,
+ @JsonProperty("rootNamespace") String rootNamespace,
+ @JsonProperty("validationMode") boolean validationMode) {
+
+ this.extensions = extensions == null
+ ? Collections.singletonList("dat")
+ : ImmutableList.copyOf(extensions);
+ this.rootName = rootName;
+ this.rootNamespace = rootNamespace;
+ this.schemaURI = schemaURI;
+ this.schemaFile = schemaFile;
+ // no default. Users must pick.
+ this.validationMode = validationMode;
+ }
+
+ @JsonInclude(JsonInclude.Include.NON_DEFAULT)
+ public List getExtensions() {
+ return extensions;
+ }
+
+ public String getSchemaURI() {
+ return schemaURI;
+ }
+
+ public String getSchemaFile() {
+ return schemaFile;
+ }
+
+ public String getRootName() {
+ return rootName;
+ }
+
+ public String getRootNamespace() {
+ return rootNamespace;
+ }
+
+ @JsonInclude(JsonInclude.Include.NON_DEFAULT)
+ public boolean getValidationMode() {
+ return validationMode;
+ }
+
+ public DaffodilReaderConfig getReaderConfig(DaffodilFormatPlugin plugin) {
+ return new DaffodilReaderConfig(plugin);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(schemaURI, schemaFile, validationMode, rootName, rootNamespace);
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (this == obj) {
+ return true;
+ }
+ if (obj == null || getClass() != obj.getClass()) {
+ return false;
+ }
+ DaffodilFormatConfig other = (DaffodilFormatConfig) obj;
+ return Objects.equals(schemaURI, other.schemaURI)
+ && Objects.equals(schemaFile, other.schemaFile)
+ && Objects.equals(rootName, other.rootName)
+ && Objects.equals(rootNamespace, other.rootNamespace)
+ && Objects.equals(validationMode, other.validationMode);
+ }
+
+ @Override
+ public String toString() {
+ return new PlanStringBuilder(this)
+ .field("schemaURI", schemaURI)
+ .field("schemaFile", schemaFile)
+ .field("rootName", rootName)
+ .field("rootNamespace", rootNamespace)
+ .field("validationMode", validationMode)
+ .toString();
+ }
+}
diff --git a/contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilFormatPlugin.java b/contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilFormatPlugin.java
new file mode 100644
index 00000000000..b6a74467286
--- /dev/null
+++ b/contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilFormatPlugin.java
@@ -0,0 +1,74 @@
+/*
+ * 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.common.logical.StoragePluginConfig;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.physical.impl.scan.v3.ManagedReader;
+import org.apache.drill.exec.physical.impl.scan.v3.file.FileReaderFactory;
+import org.apache.drill.exec.physical.impl.scan.v3.file.FileScanLifecycleBuilder;
+import org.apache.drill.exec.physical.impl.scan.v3.file.FileSchemaNegotiator;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.dfs.easy.EasyFormatPlugin;
+import org.apache.drill.exec.store.dfs.easy.EasySubScan;
+import org.apache.hadoop.conf.Configuration;
+
+public class DaffodilFormatPlugin extends EasyFormatPlugin {
+
+ public static final String DEFAULT_NAME = "daffodil";
+ public static final String OPERATOR_TYPE = "DAFFODIL_SUB_SCAN";
+
+ public DaffodilFormatPlugin(String name, DrillbitContext context, Configuration fsConf,
+ StoragePluginConfig storageConfig, DaffodilFormatConfig formatConfig) {
+ super(name, easyConfig(fsConf, formatConfig), context, storageConfig, formatConfig);
+ }
+
+ private static EasyFormatConfig easyConfig(Configuration fsConf,
+ DaffodilFormatConfig pluginConfig) {
+ return EasyFormatConfig.builder().readable(true).writable(false).blockSplittable(false)
+ .compressible(true).extensions(pluginConfig.getExtensions()).fsConf(fsConf)
+ .readerOperatorType(OPERATOR_TYPE).scanVersion(ScanFrameworkVersion.EVF_V2)
+ .supportsLimitPushdown(true).supportsProjectPushdown(true)
+ .defaultName(DaffodilFormatPlugin.DEFAULT_NAME).build();
+ }
+
+ @Override
+ protected void configureScan(FileScanLifecycleBuilder builder, EasySubScan scan) {
+ builder.nullType(Types.optional(MinorType.VARCHAR));
+ builder.readerFactory(new DaffodilReaderFactory(formatConfig.getReaderConfig(this), scan));
+ }
+
+ public static class DaffodilReaderFactory extends FileReaderFactory {
+ private final DaffodilReaderConfig readerConfig;
+
+ private final EasySubScan scan;
+
+ public DaffodilReaderFactory(DaffodilReaderConfig config,
+ EasySubScan scan) {
+ this.readerConfig = config;
+ this.scan = scan;
+ }
+
+ @Override
+ public ManagedReader newReader(FileSchemaNegotiator negotiator) {
+ return new DaffodilBatchReader(readerConfig, scan, negotiator);
+ }
+ }
+}
diff --git a/contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilMessageParser.java b/contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilMessageParser.java
new file mode 100644
index 00000000000..97f06d7d35e
--- /dev/null
+++ b/contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/DaffodilMessageParser.java
@@ -0,0 +1,196 @@
+/*
+ * 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
+ *
+ * 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.
+ *
+ * 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.
+ *
+ * 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.
+ *
+ * Note that the InfosetOutputter may be called many times before a processing error is detected, as
+ * Daffodil delivers result data incrementally.
+ *
+ * Validation errors do not affect the InfosetOutputter output calls, but indicate that data was
+ * detected that is invalid.
+ *
+ * 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.
+ *
+ * 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 {
+
+ private static final Logger logger = LoggerFactory.getLogger(DaffodilMessageParser.class);
+ private List diagnostics; // diagnostics.
+ private boolean isProcessingError;
+ private boolean isValidationError;
+ private InputSourceDataInputStream dis;
+ private InfosetOutputter outputter;
+ private DataProcessor dp;
+
+ /**
+ * 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.
+ *
+ * This input stream is then owned by this object and becomes part of its state.
+ *
+ * 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.
+ *
+ * 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.
+ *
+ * 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.
+ *
+ * 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.
+ *
+ * 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 getDiagnostics() {
+ return diagnostics;
+ }
+
+ public String getDiagnosticsAsString() {
+ String result = diagnostics.stream().map(Diagnostic::getMessage)
+ .collect(Collectors.joining("\n"));
+ return result;
+ }
+
+ private void reset() {
+ outputter.reset();
+ isProcessingError = false;
+ isValidationError = false;
+ diagnostics = null;
+ }
+}
diff --git a/contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/schema/DaffodilDataProcessorFactory.java b/contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/schema/DaffodilDataProcessorFactory.java
new file mode 100644
index 00000000000..e338f266b7e
--- /dev/null
+++ b/contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/schema/DaffodilDataProcessorFactory.java
@@ -0,0 +1,165 @@
+/*
+ * 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.
+ *
+ * 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 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 compileSchema(URI schemaFileURI, String rootName, String rootNS)
+ throws URISyntaxException, IOException, CompileFailure {
+ Compiler c = Daffodil.compiler();
+ ProcessorFactory pf = c.compileSource(schemaFileURI, rootName, rootNS);
+ List 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 pfDiags) throws CompileFailure {
+ Objects.requireNonNull(dp); // true because failure to produce a dp throws CompileFailure.
+ if (validationMode) {
+ try {
+ // We don't have the DFDL schema text, and we're not creating XML as an intermediate form,
+ // we're taking data direct from Daffodil into Drill rows, so using any Xerces-based
+ // XML Validator is not possible.
+ dp = dp.withValidationMode(ValidationMode.Limited);
+ } catch (InvalidUsageException e) {
+ // impossible
+ throw new Error(e);
+ }
+ }
+ List dpDiags = dp.getDiagnostics();
+ if (dp.isError()) {
+ throw new CompileFailure(dpDiags);
+ }
+ // well this part is only if we compiled, and provided the pfDiags arg as non-null.
+ List compilationWarnings;
+ if (pfDiags != null && !pfDiags.isEmpty()) {
+ compilationWarnings = pfDiags;
+ compilationWarnings.addAll(dpDiags); // dpDiags might be empty. That's ok.
+ } else {
+ compilationWarnings = dpDiags; // dpDiags might be empty. That's ok.
+ }
+ }
+
+ /**
+ * Thrown if schema compilation fails.
+ *
+ * Contains diagnostic objects which give the cause(s) of the failure, or
+ * contains a cause Throwable providing the reason.
+ */
+ public static class CompileFailure extends Exception {
+ List diags;
+
+ CompileFailure(List diagnostics) {
+ super("DFDL Schema Compile Failure");
+ diags = diagnostics;
+ }
+ CompileFailure(Throwable cause) {
+ super(cause);
+ }
+ }
+}
diff --git a/contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/schema/DrillDaffodilSchemaUtils.java b/contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/schema/DrillDaffodilSchemaUtils.java
new file mode 100644
index 00000000000..5481f1da7ad
--- /dev/null
+++ b/contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/schema/DrillDaffodilSchemaUtils.java
@@ -0,0 +1,114 @@
+/*
+ * 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.DFDLPrimType;
+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 DFDL_TYPE_MAPPINGS =
+ ImmutableMap.builder()
+ .put(DFDLPrimType.Long, MinorType.BIGINT)
+ .put(DFDLPrimType.Int, MinorType.INT)
+ .put(DFDLPrimType.Short, MinorType.SMALLINT)
+ .put(DFDLPrimType.Byte, MinorType.TINYINT)
+ // daffodil unsigned longs are modeled as DECIMAL(38, 0) which is the default for VARDECIMAL
+ .put(DFDLPrimType.UnsignedLong, MinorType.VARDECIMAL)
+ .put(DFDLPrimType.UnsignedInt, MinorType.BIGINT)
+ .put(DFDLPrimType.UnsignedShort, MinorType.UINT2)
+ .put(DFDLPrimType.UnsignedByte, MinorType.UINT1)
+ // daffodil integer, nonNegativeInteger, are modeled as DECIMAL(38, 0) which is the default for VARDECIMAL
+ .put(DFDLPrimType.Integer, MinorType.VARDECIMAL)
+ .put(DFDLPrimType.NonNegativeInteger, MinorType.VARDECIMAL)
+ // decimal has to be modeled as string since we really have no idea what to set the
+ // scale to.
+ .put(DFDLPrimType.Decimal, MinorType.VARCHAR)
+ .put(DFDLPrimType.Boolean, MinorType.BIT)
+ .put(DFDLPrimType.Date, MinorType.DATE) // requires conversion
+ .put(DFDLPrimType.DateTime, MinorType.TIMESTAMP) // requires conversion
+ .put(DFDLPrimType.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(DFDLPrimType.Float, MinorType.FLOAT8)
+ .put(DFDLPrimType.HexBinary, MinorType.VARBINARY)
+ .put(DFDLPrimType.String, MinorType.VARCHAR)
+ .put(DFDLPrimType.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();
+ boolean validationMode = true; // use Daffodil's limited validation always
+ DataProcessor dp = dpf.getDataProcessor(dfdlSchemaURI, validationMode, rootName, namespace);
+ return daffodilDataProcessorToDrillSchema(dp);
+ }
+
+ public static TupleMetadata daffodilDataProcessorToDrillSchema(DataProcessor dp) {
+ DrillDaffodilSchemaVisitor schemaVisitor = new DrillDaffodilSchemaVisitor();
+ dp.walkMetadata(schemaVisitor);
+ TupleMetadata drillSchema = schemaVisitor.getDrillSchema();
+ return drillSchema;
+ }
+
+ /**
+ * Returns a {@link MinorType} of the corresponding DFDL Data Type. Defaults to VARCHAR if unknown
+ * @param dfdlType The type as provided by Daffodil.
+ * @return A {@link MinorType} of the Drill data type.
+ */
+ public static MinorType getDrillDataType(DFDLPrimType dfdlType) {
+ try {
+ MinorType type = DrillDaffodilSchemaUtils.DFDL_TYPE_MAPPINGS.get(dfdlType);
+ if (type == null) {
+ return DEFAULT_TYPE;
+ } else {
+ return type;
+ }
+ } catch (NullPointerException e) {
+ logger.warn("Unknown data type found in XSD reader: {}. Returning VARCHAR.", dfdlType);
+ return DEFAULT_TYPE;
+ }
+ }
+}
diff --git a/contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/schema/DrillDaffodilSchemaVisitor.java b/contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/schema/DrillDaffodilSchemaVisitor.java
new file mode 100644
index 00000000000..0bd8992eee7
--- /dev/null
+++ b/contrib/format-daffodil/src/main/java/org/apache/drill/exec/store/daffodil/schema/DrillDaffodilSchemaVisitor.java
@@ -0,0 +1,128 @@
+/*
+ * 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.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.MapBuilderLike;
+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);
+
+ /**
+ * SchemaBuilder and MapBuilder share a polymorphic interface MapBuilderLike
+ */
+ private final SchemaBuilder builder = new SchemaBuilder();
+ private final Stack mapBuilderStack = new Stack<>();
+
+ private MapBuilderLike mapBuilder() {
+ return mapBuilderStack.peek();
+ }
+
+ /**
+ * Converts Daffodil names into appropriate Drill column names.
+ * @param md Daffodil element metadata, which contains an element name.
+ * @return a string usable as a Drill column name
+ */
+ public static String makeColumnName(ElementMetadata md) {
+ return md.toQName().replace(":", "_");
+ }
+
+ /**
+ * 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 simpleElementMetadata(SimpleElementMetadata md) {
+ assert (!mapBuilderStack.isEmpty());
+ String colName = makeColumnName(md);
+ MinorType drillType = DrillDaffodilSchemaUtils.getDrillDataType(md.dfdlType());
+ if (md.isArray()) {
+ mapBuilder().addArray(colName, drillType);
+ } else if (md.isOptional() || md.isNillable()) {
+ mapBuilder().addNullable(colName, drillType);
+ } else {
+ mapBuilder().add(colName, drillType);
+ }
+ }
+
+ @Override
+ public void startComplexElementMetadata(ComplexElementMetadata md) {
+ if (mapBuilderStack.isEmpty()) {
+ // root element case. The SchemaBuilder top level row is the container of the root element's children
+ mapBuilderStack.push(builder);
+ } else {
+ // enclosed complex element case. Create a map field.
+ String colName = makeColumnName(md);
+ if (md.isArray()) {
+ mapBuilderStack.push(mapBuilder().addMapArray(colName));
+ } else {
+ mapBuilderStack.push(mapBuilder().addMap(colName)); // also handles optional complex elements
+ }
+ }
+ }
+
+ @Override
+ public void endComplexElementMetadata(ComplexElementMetadata md) {
+ assert (!mapBuilderStack.isEmpty());
+ mapBuilder().resume();
+ mapBuilderStack.pop();
+ }
+
+ @Override
+ public void startSequenceMetadata(SequenceMetadata m) {
+ }
+
+ @Override
+ public void endSequenceMetadata(SequenceMetadata m) {
+ }
+
+ @Override
+ public void startChoiceMetadata(ChoiceMetadata m) {
+ }
+
+ @Override
+ public void endChoiceMetadata(ChoiceMetadata m) {
+ }
+
+ private void DFDLSchemaError(String s) {
+ throw new RuntimeException(s);
+ }
+}
diff --git a/contrib/format-daffodil/src/main/resources/bootstrap-format-plugins.json b/contrib/format-daffodil/src/main/resources/bootstrap-format-plugins.json
new file mode 100644
index 00000000000..966d9ba1b5b
--- /dev/null
+++ b/contrib/format-daffodil/src/main/resources/bootstrap-format-plugins.json
@@ -0,0 +1,37 @@
+{
+ "storage":{
+ "dfs": {
+ "type": "file",
+ "formats": {
+ "daffodil": {
+ "type": "daffodil",
+ "extensions": [
+ "dat"
+ ]
+ }
+ }
+ },
+ "cp": {
+ "type": "file",
+ "formats": {
+ "daffodil": {
+ "type": "daffodil",
+ "extensions": [
+ "dat"
+ ]
+ }
+ }
+ },
+ "s3": {
+ "type": "file",
+ "formats": {
+ "daffodil": {
+ "type": "daffodil",
+ "extensions": [
+ "dat"
+ ]
+ }
+ }
+ }
+ }
+}
diff --git a/contrib/format-daffodil/src/main/resources/drill-module.conf b/contrib/format-daffodil/src/main/resources/drill-module.conf
new file mode 100644
index 00000000000..52a902572e3
--- /dev/null
+++ b/contrib/format-daffodil/src/main/resources/drill-module.conf
@@ -0,0 +1,25 @@
+#
+# 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.
+#
+
+# This file tells Drill to consider this module when class path scanning.
+# This file can also include any supplementary configuration information.
+# This file is in HOCON format, see https://github.com/typesafehub/config/blob/master/HOCON.md for more information.
+
+drill.classpath.scanning: {
+ packages += "org.apache.drill.exec.store.daffodil"
+}
diff --git a/contrib/format-daffodil/src/test/java/org/apache/drill/exec/store/daffodil/TestDaffodilReader.java b/contrib/format-daffodil/src/test/java/org/apache/drill/exec/store/daffodil/TestDaffodilReader.java
new file mode 100644
index 00000000000..17458fa62fc
--- /dev/null
+++ b/contrib/format-daffodil/src/test/java/org/apache/drill/exec/store/daffodil/TestDaffodilReader.java
@@ -0,0 +1,258 @@
+/*
+ * 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.net.URISyntaxException;
+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 {
+
+ final String SCHEMA_URI_ROOT = getClass()
+ .getClassLoader()
+ .getResource("schema/")
+ .toURI()
+ .toString();
+
+ public TestDaffodilReader() throws URISyntaxException {
+ }
+
+ @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 => '" + SCHEMA_URI_ROOT + 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();
+ 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();
+ 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();
+ 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();
+ assertEquals(3, results.rowCount());
+
+ RowSetReader rdr = results.reader();
+ rdr.next();
+ String map = rdr.getAsString();
+ assertEquals("{257, 258}", map);
+ rdr.next();
+ map = rdr.getAsString();
+ assertEquals("{259, 260}", map);
+ rdr.next();
+ map = rdr.getAsString();
+ assertEquals("{261, 262}", map);
+ assertFalse(rdr.next());
+ results.clear();
+ }
+
+ /**
+ * Tests data which is rows of two ints and an array containing a map containing two ints. Each
+ * row can be visualized like this: "{257, 258, [{259, 260},...]}"
+ */
+ @Test
+ public void testComplexArrayQuery1() throws Exception {
+
+ QueryBuilder qb = client.queryBuilder();
+ QueryBuilder query = qb.sql(selectRow("complexArray1", "data12Int.dat"));
+ RowSet results = query.rowSet();
+ assertEquals(1, results.rowCount());
+
+ RowSetReader rdr = results.reader();
+ rdr.next();
+ String map = rdr.getAsString();
+ assertEquals("{257, 258, [{259, 260}, {261, 262}, {257, 258}, {259, 260}, {261, 262}]}", map);
+ assertFalse(rdr.next());
+ results.clear();
+ }
+
+ /**
+ * Tests data which is an array of ints in one column of the row set
+ */
+ @Test
+ public void testSimpleArrayQuery1() throws Exception {
+
+ QueryBuilder qb = client.queryBuilder();
+ QueryBuilder query = qb.sql(selectRow("simpleArrayField1", "data12Int.dat"));
+ RowSet results = query.rowSet();
+ assertEquals(1, results.rowCount());
+
+ RowSetReader rdr = results.reader();
+ rdr.next();
+ String map = rdr.getAsString();
+ assertEquals("{[257, 258, 259, 260, 261, 262, 257, 258, 259, 260, 261, 262]}", map);
+ assertFalse(rdr.next());
+ results.clear();
+ }
+
+ /**
+ * Tests data which is rows of two ints and an array containing a map containing an int and a
+ * vector of ints. Each row can be visualized like this: "{257, 258, [{259, [260, 261,
+ * 262]},...]}"
+ */
+ @Test
+ public void testComplexArrayQuery2() throws Exception {
+
+ QueryBuilder qb = client.queryBuilder();
+ QueryBuilder query = qb.sql(selectRow("complexArray2", "data12Int.dat"));
+ RowSet results = query.rowSet();
+ assertEquals(1, results.rowCount());
+
+ RowSetReader rdr = results.reader();
+ rdr.next();
+ String map = rdr.getAsString();
+ assertEquals("{257, 258, [{259, [260, 261, 262]}, {257, [258, 259, 260]}, {261, [262]}]}", map);
+ assertFalse(rdr.next());
+ results.clear();
+ }
+
+ @Test
+ public void testMoreTypes1() throws Exception {
+
+ QueryBuilder qb = client.queryBuilder();
+ QueryBuilder query = qb.sql(selectRow("moreTypes1", "moreTypes1.txt.dat"));
+ RowSet results = query.rowSet();
+ assertEquals(2, results.rowCount());
+
+ RowSetReader rdr = results.reader();
+ rdr.next();
+ String map = rdr.getAsString();
+ assertEquals(
+ "{2147483647, 9223372036854775807, 32767, 127, true, " + "1.7976931348623157E308, 3" +
+ ".4028235E38, [31, 32, 33, 34, 35, 36, 37, 38], \"daffodil\"}",
+ map);
+ rdr.next();
+ map = rdr.getAsString();
+ assertEquals(
+ "{-2147483648, -9223372036854775808, -32768, -128, false, " + "-1.7976931348623157E308, " +
+ "-3.4028235E38, [38, 37, 36, 35, 34, 33, 32, 31], \"drill\"}",
+ map);
+ assertFalse(rdr.next());
+ results.clear();
+ }
+
+ @Test
+ public void testMoreTypes2() throws Exception {
+
+ QueryBuilder qb = client.queryBuilder();
+ QueryBuilder query = qb.sql(selectRow("moreTypes2", "moreTypes2.txt.dat"));
+ RowSet results = query.rowSet();
+ assertEquals(1, results.rowCount());
+
+ RowSetReader rdr = results.reader();
+ rdr.next();
+ String map = rdr.getAsString();
+ assertEquals(
+ "{4294967295, 18446744073709551615, 65535, 255, " + "-18446744073709551616, " +
+ "18446744073709551616, " + "\"0.18446744073709551616\", " + // xs:decimal is modeled
+ // as VARCHAR i.e., a string. So needs quotation marks.
+ "1970-01-01, 00:00, 1970-01-01T00:00:00Z}", map);
+ assertFalse(rdr.next());
+ results.clear();
+ }
+}
diff --git a/contrib/format-daffodil/src/test/java/org/apache/drill/exec/store/daffodil/TestDaffodilSchemaRegistrationIntegration.java b/contrib/format-daffodil/src/test/java/org/apache/drill/exec/store/daffodil/TestDaffodilSchemaRegistrationIntegration.java
new file mode 100644
index 00000000000..bf89c4469a5
--- /dev/null
+++ b/contrib/format-daffodil/src/test/java/org/apache/drill/exec/store/daffodil/TestDaffodilSchemaRegistrationIntegration.java
@@ -0,0 +1,260 @@
+/*
+ * 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.commons.io.FileUtils;
+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.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.io.File;
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.List;
+
+import static org.apache.drill.test.HadoopUtils.hadoopToJavaPath;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Integration test for the complete Daffodil schema registration workflow.
+ * This test verifies:
+ * 1. Registering a DFDL schema using CREATE DAFFODIL SCHEMA (both JAR and XSD formats)
+ * 2. Querying data files using the registered schema
+ * 3. Unregistering the schema using DROP DAFFODIL SCHEMA
+ */
+@Category(RowSetTest.class)
+public class TestDaffodilSchemaRegistrationIntegration extends ClusterTest {
+
+ private static Path stagingArea;
+ private static File schemaResourceDir;
+
+ @BeforeClass
+ public static void setup() throws Exception {
+ // Start the test cluster
+ ClusterTest.startCluster(ClusterFixture.builder(dirTestWatcher));
+
+ // Define the Daffodil format
+ DaffodilFormatConfig formatConfig = new DaffodilFormatConfig(List.of("dat"), "", "", "", "", false);
+ cluster.defineFormat("dfs", "daffodil", formatConfig);
+
+ // Copy test data and schema files to the test directory
+ dirTestWatcher.copyResourceToRoot(Paths.get("data/"));
+ dirTestWatcher.copyResourceToRoot(Paths.get("schema/"));
+
+ // Get the staging area for schema files
+ stagingArea = hadoopToJavaPath(cluster.drillbit().getContext()
+ .getRemoteDaffodilSchemaRegistry().getStagingArea());
+
+ // Locate the schema resource directory
+ try {
+ schemaResourceDir = Paths.get(
+ TestDaffodilSchemaRegistrationIntegration.class
+ .getClassLoader()
+ .getResource("schema/")
+ .toURI()
+ ).toFile();
+ } catch (URISyntaxException e) {
+ throw new RuntimeException("Failed to locate test schema directory", e);
+ }
+ }
+
+ /**
+ * End-to-end test that:
+ * 1. Registers a DFDL schema XSD file using CREATE DAFFODIL SCHEMA
+ * 2. Queries a data file using the registered schema
+ * 3. Verifies the query results
+ */
+ @Test
+ public void testRegisterXsdSchemaAndQuery() throws Exception {
+ String schemaFileName = "simple.dfdl.xsd";
+ File sourceSchema = new File(schemaResourceDir, schemaFileName);
+
+ // Copy the schema XSD to the staging area
+ copyFileToStaging(sourceSchema, stagingArea, schemaFileName);
+
+ // Step 1: Register the schema
+ client.testBuilder()
+ .sqlQuery("CREATE DAFFODIL SCHEMA USING JAR '%s'", schemaFileName)
+ .unOrdered()
+ .baselineColumns("ok", "summary")
+ .baselineValues(true, String.format("Daffodil schema jar %s has been registered successfully.", schemaFileName))
+ .go();
+
+ // Verify schema was moved from staging to registry
+ File stagingFile = stagingArea.resolve(schemaFileName).toFile();
+ Path registryArea = hadoopToJavaPath(cluster.drillbit().getContext()
+ .getRemoteDaffodilSchemaRegistry().getRegistryArea());
+ File registryFile = registryArea.resolve(schemaFileName).toFile();
+
+ assertFalse("Schema file should be removed from staging after registration", stagingFile.exists());
+ assertTrue("Schema file should exist in registry after registration", registryFile.exists());
+
+ // Step 2: Query data using the registered schema
+ // After CREATE DAFFODIL SCHEMA, the schema file is moved from staging to the registry area
+ // Use schemaFile parameter (just the filename) - Drill will automatically look it up in the registry
+ String query = "SELECT * FROM table(dfs.`data/data01Int.dat` " +
+ "(type => 'daffodil', " +
+ "validationMode => 'true', " +
+ "schemaFile => '" + schemaFileName + "', " +
+ "rootName => 'row', " +
+ "rootNamespace => null))";
+
+ QueryBuilder qb = client.queryBuilder();
+ RowSet results = qb.sql(query).rowSet();
+
+ // Step 3: Verify results
+ assertEquals(1, results.rowCount());
+
+ TupleMetadata expectedSchema = new SchemaBuilder()
+ .add("col", MinorType.INT)
+ .buildSchema();
+
+ RowSet expected = client.rowSetBuilder(expectedSchema)
+ .addRow(0x00000101) // aka 257
+ .build();
+
+ new RowSetComparison(expected).verifyAndClearAll(results);
+
+ // Clean up - drop the schema
+ client.testBuilder()
+ .sqlQuery("DROP DAFFODIL SCHEMA USING JAR '%s'", schemaFileName)
+ .unOrdered()
+ .baselineColumns("ok", "summary")
+ .baselineValues(true, String.format("Daffodil schema jar %s has been unregistered successfully.", schemaFileName))
+ .go();
+ }
+
+ /**
+ * Test the complete lifecycle: register, query, then unregister
+ */
+ @Test
+ public void testCompleteSchemaLifecycle() throws Exception {
+ String schemaFileName = "complex1.dfdl.xsd";
+ File sourceSchema = new File(schemaResourceDir, schemaFileName);
+
+ // Copy to staging area
+ copyFileToStaging(sourceSchema, stagingArea, schemaFileName);
+
+ // Register the schema
+ client.testBuilder()
+ .sqlQuery("CREATE DAFFODIL SCHEMA USING JAR '%s'", schemaFileName)
+ .unOrdered()
+ .baselineColumns("ok", "summary")
+ .baselineValues(true, String.format("Daffodil schema jar %s has been registered successfully.", schemaFileName))
+ .go();
+
+ // Unregister the schema
+ client.testBuilder()
+ .sqlQuery("DROP DAFFODIL SCHEMA USING JAR '%s'", schemaFileName)
+ .unOrdered()
+ .baselineColumns("ok", "summary")
+ .baselineValues(true, String.format("Daffodil schema jar %s has been unregistered successfully.", schemaFileName))
+ .go();
+
+ // Verify the file is removed from the registry
+ Path registryArea = hadoopToJavaPath(cluster.drillbit().getContext()
+ .getRemoteDaffodilSchemaRegistry().getRegistryArea());
+ File registeredFile = registryArea.resolve(schemaFileName).toFile();
+ assertTrue("Schema file should be removed from registry after dropping", !registeredFile.exists());
+ }
+
+ /**
+ * Test querying with multiple rows of data using a registered schema
+ */
+ @Test
+ public void testQueryMultipleRowsWithRegisteredSchema() throws Exception {
+ String schemaFileName = "simple.dfdl.xsd";
+ File sourceSchema = new File(schemaResourceDir, schemaFileName);
+
+ // Copy to staging area and register
+ copyFileToStaging(sourceSchema, stagingArea, schemaFileName);
+
+ client.testBuilder()
+ .sqlQuery("CREATE DAFFODIL SCHEMA USING JAR '%s'", schemaFileName)
+ .unOrdered()
+ .baselineColumns("ok", "summary")
+ .baselineValues(true, String.format("Daffodil schema jar %s has been registered successfully.", schemaFileName))
+ .go();
+
+ // Query data with 6 rows using the registered schema
+ // Use schemaFile parameter (just the filename) - Drill will automatically look it up in the registry
+ String query = "SELECT * FROM table(dfs.`data/data06Int.dat` " +
+ "(type => 'daffodil', " +
+ "validationMode => 'true', " +
+ "schemaFile => '" + schemaFileName + "', " +
+ "rootName => 'row', " +
+ "rootNamespace => null))";
+
+ QueryBuilder qb = client.queryBuilder();
+ RowSet results = qb.sql(query).rowSet();
+
+ // Verify results
+ assertEquals(6, results.rowCount());
+
+ 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);
+
+ // Clean up - drop the schema
+ client.testBuilder()
+ .sqlQuery("DROP DAFFODIL SCHEMA USING JAR '%s'", schemaFileName)
+ .unOrdered()
+ .baselineColumns("ok", "summary")
+ .baselineValues(true, String.format("Daffodil schema jar %s has been unregistered successfully.", schemaFileName))
+ .go();
+ }
+
+ /**
+ * Helper method to copy a file to the staging area
+ */
+ private void copyFileToStaging(File sourceFile, Path destination, String fileName) throws IOException {
+ File destFile = destination.resolve(fileName).toFile();
+
+ // Ensure the staging directory exists
+ if (!destination.toFile().exists()) {
+ destination.toFile().mkdirs();
+ }
+
+ FileUtils.copyFile(sourceFile, destFile);
+ }
+}
diff --git a/contrib/format-daffodil/src/test/java/org/apache/drill/exec/store/daffodil/TestDynamicSchemata.java b/contrib/format-daffodil/src/test/java/org/apache/drill/exec/store/daffodil/TestDynamicSchemata.java
new file mode 100644
index 00000000000..f5f657f0250
--- /dev/null
+++ b/contrib/format-daffodil/src/test/java/org/apache/drill/exec/store/daffodil/TestDynamicSchemata.java
@@ -0,0 +1,31 @@
+/*
+ * 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.test.BaseTestQuery;
+import org.junit.Test;
+
+public class TestDynamicSchemata extends BaseTestQuery {
+
+ @Test
+ public void testSchema() throws Exception {
+ test("CREATE DAFFODIL SCHEMA USING JAR 'xxx.jar'");
+
+ }
+}
diff --git a/contrib/format-daffodil/src/test/java/org/apache/drill/exec/store/daffodil/schema/TestDaffodilToDrillMetadataConversion.java b/contrib/format-daffodil/src/test/java/org/apache/drill/exec/store/daffodil/schema/TestDaffodilToDrillMetadataConversion.java
new file mode 100644
index 00000000000..8076fe5e8cf
--- /dev/null
+++ b/contrib/format-daffodil/src/test/java/org/apache/drill/exec/store/daffodil/schema/TestDaffodilToDrillMetadataConversion.java
@@ -0,0 +1,71 @@
+/*
+ * 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.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.junit.Test;
+
+import java.net.URI;
+
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class TestDaffodilToDrillMetadataConversion {
+
+ @Test
+ public void testSimple() throws Exception {
+ URI schemaURI = getClass().getResource("/schema/simple.dfdl.xsd").toURI();
+ TupleMetadata schema = DrillDaffodilSchemaUtils.processSchema(schemaURI, "row", null);
+
+ TupleMetadata expectedSchema = new SchemaBuilder()
+ .add("col", MinorType.INT)
+ .buildSchema();
+ assertTrue(expectedSchema.isEquivalent(schema));
+ }
+
+ @Test
+ public void testComplex1() throws Exception {
+ URI schemaURI = getClass().getResource("/schema/complex1.dfdl.xsd").toURI();
+ TupleMetadata schema = DrillDaffodilSchemaUtils.processSchema(schemaURI, "row", null);
+
+ TupleMetadata expectedSchema = new SchemaBuilder()
+ .add("a1", MinorType.INT)
+ .add("a2", MinorType.INT)
+ .buildSchema();
+ assertTrue(expectedSchema.isEquivalent(schema));
+ }
+
+ @Test
+ public void testComplex2() throws Exception {
+ URI schemaURI = getClass().getResource("/schema/complex2.dfdl.xsd").toURI();
+ TupleMetadata schema = DrillDaffodilSchemaUtils.processSchema(schemaURI, "row", null);
+
+ TupleMetadata expectedSchema = new SchemaBuilder()
+ .add("a1", MinorType.INT)
+ .add("a2", MinorType.INT)
+ .addMap("b")
+ .add("b1", MinorType.INT)
+ .add("b2", MinorType.INT)
+ .resumeSchema()
+ .buildSchema();
+ assertTrue(expectedSchema.isEquivalent(schema));
+ }
+
+}
diff --git a/contrib/format-daffodil/src/test/resources/data/data01Int.dat b/contrib/format-daffodil/src/test/resources/data/data01Int.dat
new file mode 100644
index 00000000000..dee9c4c8ada
Binary files /dev/null and b/contrib/format-daffodil/src/test/resources/data/data01Int.dat differ
diff --git a/contrib/format-daffodil/src/test/resources/data/data01Int.dat.gz b/contrib/format-daffodil/src/test/resources/data/data01Int.dat.gz
new file mode 100644
index 00000000000..5e4b3b37acf
Binary files /dev/null and b/contrib/format-daffodil/src/test/resources/data/data01Int.dat.gz differ
diff --git a/contrib/format-daffodil/src/test/resources/data/data02Int.dat b/contrib/format-daffodil/src/test/resources/data/data02Int.dat
new file mode 100644
index 00000000000..8577a259181
Binary files /dev/null and b/contrib/format-daffodil/src/test/resources/data/data02Int.dat differ
diff --git a/contrib/format-daffodil/src/test/resources/data/data06Int.dat b/contrib/format-daffodil/src/test/resources/data/data06Int.dat
new file mode 100644
index 00000000000..8c29db2ec51
Binary files /dev/null and b/contrib/format-daffodil/src/test/resources/data/data06Int.dat differ
diff --git a/contrib/format-daffodil/src/test/resources/data/data12Int.dat b/contrib/format-daffodil/src/test/resources/data/data12Int.dat
new file mode 100644
index 00000000000..39fa5271b4f
Binary files /dev/null and b/contrib/format-daffodil/src/test/resources/data/data12Int.dat differ
diff --git a/contrib/format-daffodil/src/test/resources/data/moreTypes1.txt.dat b/contrib/format-daffodil/src/test/resources/data/moreTypes1.txt.dat
new file mode 100644
index 00000000000..cd0b5c1591a
--- /dev/null
+++ b/contrib/format-daffodil/src/test/resources/data/moreTypes1.txt.dat
@@ -0,0 +1,2 @@
+2147483647 9223372036854775807 32767 127 T 1.7976931348623157E308 3.4028235E38 12345678 'daffodil'
+-2147483648 -9223372036854775808 -32768 -128 F -1.7976931348623157E308 -3.4028235E38 87654321 'drill'
diff --git a/contrib/format-daffodil/src/test/resources/data/moreTypes2.txt.dat b/contrib/format-daffodil/src/test/resources/data/moreTypes2.txt.dat
new file mode 100644
index 00000000000..fca536e8543
--- /dev/null
+++ b/contrib/format-daffodil/src/test/resources/data/moreTypes2.txt.dat
@@ -0,0 +1 @@
+4294967295 18446744073709551615 65535 255 -18446744073709551616 18446744073709551616 0.18446744073709551616 1970-01-01 00:00:00+0000 1970-01-01T00:00:00
diff --git a/contrib/format-daffodil/src/test/resources/schema-jars/simple-test-schema.jar b/contrib/format-daffodil/src/test/resources/schema-jars/simple-test-schema.jar
new file mode 100644
index 00000000000..c13806066ca
Binary files /dev/null and b/contrib/format-daffodil/src/test/resources/schema-jars/simple-test-schema.jar differ
diff --git a/contrib/format-daffodil/src/test/resources/schema/complex1.dfdl.xsd b/contrib/format-daffodil/src/test/resources/schema/complex1.dfdl.xsd
new file mode 100644
index 00000000000..1f4ab8954ef
--- /dev/null
+++ b/contrib/format-daffodil/src/test/resources/schema/complex1.dfdl.xsd
@@ -0,0 +1,54 @@
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
diff --git a/contrib/format-daffodil/src/test/resources/schema/complex2.dfdl.xsd b/contrib/format-daffodil/src/test/resources/schema/complex2.dfdl.xsd
new file mode 100644
index 00000000000..d4d74aefe1f
--- /dev/null
+++ b/contrib/format-daffodil/src/test/resources/schema/complex2.dfdl.xsd
@@ -0,0 +1,65 @@
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
diff --git a/contrib/format-daffodil/src/test/resources/schema/complexArray1.dfdl.xsd b/contrib/format-daffodil/src/test/resources/schema/complexArray1.dfdl.xsd
new file mode 100644
index 00000000000..59d285bd31e
--- /dev/null
+++ b/contrib/format-daffodil/src/test/resources/schema/complexArray1.dfdl.xsd
@@ -0,0 +1,65 @@
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
diff --git a/contrib/format-daffodil/src/test/resources/schema/complexArray2.dfdl.xsd b/contrib/format-daffodil/src/test/resources/schema/complexArray2.dfdl.xsd
new file mode 100644
index 00000000000..89d74fa2de9
--- /dev/null
+++ b/contrib/format-daffodil/src/test/resources/schema/complexArray2.dfdl.xsd
@@ -0,0 +1,65 @@
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
diff --git a/contrib/format-daffodil/src/test/resources/schema/moreTypes1.dfdl.xsd b/contrib/format-daffodil/src/test/resources/schema/moreTypes1.dfdl.xsd
new file mode 100644
index 00000000000..a294e29e548
--- /dev/null
+++ b/contrib/format-daffodil/src/test/resources/schema/moreTypes1.dfdl.xsd
@@ -0,0 +1,65 @@
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
diff --git a/contrib/format-daffodil/src/test/resources/schema/moreTypes2.dfdl.xsd b/contrib/format-daffodil/src/test/resources/schema/moreTypes2.dfdl.xsd
new file mode 100644
index 00000000000..00bd3d7a155
--- /dev/null
+++ b/contrib/format-daffodil/src/test/resources/schema/moreTypes2.dfdl.xsd
@@ -0,0 +1,64 @@
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
diff --git a/contrib/format-daffodil/src/test/resources/schema/simple.dfdl.xsd b/contrib/format-daffodil/src/test/resources/schema/simple.dfdl.xsd
new file mode 100644
index 00000000000..eea582b9a56
--- /dev/null
+++ b/contrib/format-daffodil/src/test/resources/schema/simple.dfdl.xsd
@@ -0,0 +1,71 @@
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
diff --git a/contrib/format-daffodil/src/test/resources/schema/simpleArrayField1.dfdl.xsd b/contrib/format-daffodil/src/test/resources/schema/simpleArrayField1.dfdl.xsd
new file mode 100644
index 00000000000..6c72c375159
--- /dev/null
+++ b/contrib/format-daffodil/src/test/resources/schema/simpleArrayField1.dfdl.xsd
@@ -0,0 +1,71 @@
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
diff --git a/contrib/pom.xml b/contrib/pom.xml
index 50ccdc90dd0..700b3590397 100644
--- a/contrib/pom.xml
+++ b/contrib/pom.xml
@@ -42,6 +42,7 @@
data
format-access
+ format-daffodil
format-deltalake
format-esri
format-excel
diff --git a/distribution/pom.xml b/distribution/pom.xml
index 23119c241ed..d64ae7a26f9 100644
--- a/distribution/pom.xml
+++ b/distribution/pom.xml
@@ -511,6 +511,11 @@
drill-format-log
${project.version}
+
+ org.apache.drill.contrib
+ drill-format-daffodil
+ ${project.version}
+
org.apache.drill.contrib
drill-druid-storage
diff --git a/distribution/src/assemble/component.xml b/distribution/src/assemble/component.xml
index 66792fd43f6..76ba62978fe 100644
--- a/distribution/src/assemble/component.xml
+++ b/distribution/src/assemble/component.xml
@@ -29,6 +29,7 @@
org.apache.drill.contrib.data:tpch-sample-data:jar
org.apache.drill.contrib:drill-deltalake-format:jar
org.apache.drill.contrib:drill-druid-storage:jar
+ org.apache.drill.contrib:drill-format-daffodil:jar
org.apache.drill.contrib:drill-format-esri:jar
org.apache.drill.contrib:drill-format-excel:jar
org.apache.drill.contrib:drill-format-hdf5:jar
diff --git a/distribution/src/main/resources/drill-override-example.conf b/distribution/src/main/resources/drill-override-example.conf
index e9dde901e78..a090c70346d 100644
--- a/distribution/src/main/resources/drill-override-example.conf
+++ b/distribution/src/main/resources/drill-override-example.conf
@@ -322,6 +322,21 @@ drill.exec: {
root: "/app/drill"
}
},
+
+ # Settings for Apache Daffodil schemata.
+ daffodil: {
+ # number of retry attempts to update remote function registry
+ # if registry version was changed during update
+ retry-attempts: 10,
+ directory: {
+ # Override this property if custom file system should be used to create remote directories
+ # instead of default taken from Hadoop configuration
+ fs: "hdfs:///",
+ # Set this property if custom absolute root should be used for remote directories
+ root: "/app/drill"
+ }
+ },
+
# Settings for Temporary Tables (see https://issues.apache.org/jira/browse/DRILL-4956 for details).
# Temporary table can be created ONLY in default temporary workspace.
# Full workspace name should be indicated (including schema and workspace separated by dot).
diff --git a/exec/java-exec/src/main/codegen/data/Parser.tdd b/exec/java-exec/src/main/codegen/data/Parser.tdd
index 98107cd44c3..5d922d91db4 100644
--- a/exec/java-exec/src/main/codegen/data/Parser.tdd
+++ b/exec/java-exec/src/main/codegen/data/Parser.tdd
@@ -30,6 +30,7 @@
# List of keywords.
keywords: [
+ "DAFFODIL",
"DATABASES",
"SCHEMAS",
"TABLES",
@@ -67,6 +68,8 @@
"SqlRefreshMetadata()",
"SqlCreateFunction()",
"SqlDropFunction()",
+ "SqlCreateDaffodilSchema()",
+ "SqlDropDaffodilSchema()",
"SqlAnalyzeTable()",
"DrillSqlSetOption(Span.of(), null)",
"DrillSqlResetOption(Span.of(), null)",
diff --git a/exec/java-exec/src/main/codegen/includes/parserImpls.ftl b/exec/java-exec/src/main/codegen/includes/parserImpls.ftl
index bb3b3c7b19e..ee9eadaab26 100644
--- a/exec/java-exec/src/main/codegen/includes/parserImpls.ftl
+++ b/exec/java-exec/src/main/codegen/includes/parserImpls.ftl
@@ -687,6 +687,48 @@ SqlNode SqlDropFunction() :
}
}
+/**
+* Parse create Daffodil Schema statement
+* CREATE DAFFODIL SCHEMA USING JAR 'jar_name'
+*/
+SqlNode SqlCreateDaffodilSchema() :
+{
+ SqlParserPos pos;
+ SqlNode jar;
+}
+{
+ { pos = getPos(); }
+
+
+
+
+ jar = StringLiteral()
+ {
+ return new SqlCreateDaffodilSchema(pos, jar);
+ }
+ }
+
+/**
+* Parse drop UDF statement
+* DROP DAFFODIL SCHEMA USING JAR 'jar_name'
+*/
+SqlNode SqlDropDaffodilSchema() :
+{
+SqlParserPos pos;
+SqlNode jar;
+}
+{
+ { pos = getPos(); }
+
+
+
+
+ jar = StringLiteral()
+ {
+ return new SqlDropDaffodilSchema(pos, jar);
+ }
+}
+
/**
* Parses a analyze statements:
*
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
index 77da9092036..ec31737b073 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
@@ -17,7 +17,6 @@
*/
package org.apache.drill.exec;
-import java.util.Arrays;
import org.apache.drill.common.config.DrillConfig;
import org.apache.drill.exec.physical.impl.common.HashTable;
import org.apache.drill.exec.rpc.user.InboundImpersonationManager;
@@ -42,6 +41,8 @@
import org.apache.drill.exec.testing.ExecutionControls;
import org.apache.drill.exec.vector.ValueVector;
+import java.util.Arrays;
+
public final class ExecConstants {
private ExecConstants() {
// Don't allow instantiation
@@ -343,6 +344,17 @@ private ExecConstants() {
public static final String UDF_DIRECTORY_TMP = "drill.exec.udf.directory.tmp";
public static final String UDF_DISABLE_DYNAMIC = "drill.exec.udf.disable_dynamic";
+ /**
+ * Configuration properties for Apache Daffodil
+ */
+ public static final String DFDL_RETRY_ATTEMPTS = "drill.exec.daffodil.retry-attempts";
+ public static final String DFDL_DIRECTORY_LOCAL = "drill.exec.daffodil.directory.local";
+ public static final String DFDL_DIRECTORY_FS = "drill.exec.daffodil.directory.fs";
+ public static final String DFDL_DIRECTORY_ROOT = "drill.exec.daffodil.directory.root";
+ public static final String DFDL_DIRECTORY_STAGING = "drill.exec.daffodil.directory.staging";
+ public static final String DFDL_DIRECTORY_REGISTRY = "drill.exec.daffodil.directory.registry";
+ public static final String DFDL_DIRECTORY_TMP = "drill.exec.daffodil.directory.tmp";
+
/**
* Local temporary directory is used as base for temporary storage of Dynamic UDF jars.
*/
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
index 7d6309eb016..90cb45e4138 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
@@ -17,10 +17,10 @@
*/
package org.apache.drill.exec.ops;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-
+import com.google.common.base.Function;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import io.netty.buffer.DrillBuf;
import org.apache.calcite.schema.SchemaPlus;
import org.apache.drill.common.AutoCloseables;
import org.apache.drill.common.config.DrillConfig;
@@ -35,6 +35,7 @@
import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
import org.apache.drill.exec.physical.resultSet.impl.ResultSetLoaderImpl;
import org.apache.drill.exec.physical.resultSet.impl.ResultSetLoaderImpl.ResultSetOptions;
+import org.apache.drill.exec.schema.daffodil.RemoteDaffodilSchemaRegistry;
import org.apache.drill.exec.planner.physical.PlannerSettings;
import org.apache.drill.exec.planner.sql.DrillOperatorTable;
import org.apache.drill.exec.proto.BitControl.QueryContextInformation;
@@ -56,13 +57,11 @@
import org.apache.drill.exec.store.StoragePluginRegistry;
import org.apache.drill.exec.testing.ExecutionControls;
import org.apache.drill.exec.util.Utilities;
-
import org.apache.drill.metastore.MetastoreRegistry;
-import com.google.common.base.Function;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import io.netty.buffer.DrillBuf;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
// TODO - consider re-name to PlanningContext, as the query execution context actually appears
// in fragment contexts
@@ -215,8 +214,8 @@ public SchemaPlus getRootSchema(SchemaConfig schemaConfig) {
}
/**
- * Get the user name of the user who issued the query that is managed by this QueryContext.
- * @return The user name of the user who issued the query that is managed by this QueryContext.
+ * Get the username of the user who issued the query that is managed by this QueryContext.
+ * @return The username of the user who issued the query that is managed by this QueryContext.
*/
@Override
public String getQueryUserName() {
@@ -319,6 +318,10 @@ public RemoteFunctionRegistry getRemoteFunctionRegistry() {
return drillbitContext.getRemoteFunctionRegistry();
}
+ public RemoteDaffodilSchemaRegistry getDaffodilSchemaRegistry() {
+ return drillbitContext.getRemoteDaffodilSchemaRegistry();
+ }
+
@Override
public ContextInformation getContextInformation() {
return contextInformation;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/CreateDaffodilSchemaHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/CreateDaffodilSchemaHandler.java
new file mode 100644
index 00000000000..8ecf5946f51
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/CreateDaffodilSchemaHandler.java
@@ -0,0 +1,262 @@
+/*
+ * 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.planner.sql.handlers;
+
+import com.google.common.collect.Lists;
+import org.apache.calcite.sql.SqlCharStringLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.drill.common.exceptions.DrillRuntimeException;
+import org.apache.drill.exec.exception.JarValidationException;
+import org.apache.drill.exec.exception.VersionMismatchException;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.planner.sql.DirectPlan;
+import org.apache.drill.exec.planner.sql.parser.SqlCreateDaffodilSchema;
+import org.apache.drill.exec.proto.UserBitShared.Jar;
+import org.apache.drill.exec.proto.UserBitShared.Registry;
+import org.apache.drill.exec.schema.daffodil.RemoteDaffodilSchemaRegistry;
+import org.apache.drill.exec.store.sys.store.DataChangeVersion;
+import org.apache.drill.exec.work.foreman.ForemanSetupException;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.UUID;
+
+public class CreateDaffodilSchemaHandler extends DefaultSqlHandler {
+ private static Logger logger = LoggerFactory.getLogger(CreateDaffodilSchemaHandler.class);
+
+ public CreateDaffodilSchemaHandler(SqlHandlerConfig config) {
+ super(config);
+ }
+
+ /**
+ * Registers Daffodil schema JARs dynamically. Process consists of several steps:
+ *
+ * - Registering jar in jar registry to ensure that several jars with the same name is not registered.
+ * - Schema jar validation and back up.
+ * - Validation against remote schema registry.
+ * - Remote schema registry update.
+ * - Copying of jar to registry area and clean up.
+ *
+ *
+ * @return - Single row indicating successful registration, or error message otherwise.
+ */
+ @Override
+ public PhysicalPlan getPlan(SqlNode sqlNode) throws ForemanSetupException, IOException {
+ RemoteDaffodilSchemaRegistry remoteRegistry = context.getDaffodilSchemaRegistry();
+ JarManager jarManager = new JarManager(sqlNode, remoteRegistry);
+
+ boolean inProgress = false;
+ try {
+ final String action = remoteRegistry.addToJars(jarManager.getJarName(), RemoteDaffodilSchemaRegistry.Action.REGISTRATION);
+ if (!(inProgress = action == null)) {
+ return DirectPlan.createDirectPlan(context, false,
+ String.format("Jar with %s name is used. Action: %s", jarManager.getJarName(), action));
+ }
+
+ jarManager.initRemoteBackup();
+ initRemoteRegistration(jarManager, remoteRegistry);
+ jarManager.deleteQuietlyFromStagingArea();
+
+ return DirectPlan.createDirectPlan(context, true,
+ String.format("Daffodil schema jar %s has been registered successfully.", jarManager.getJarName()));
+
+ } catch (Exception e) {
+ logger.error("Error during Daffodil schema registration", e);
+ return DirectPlan.createDirectPlan(context, false, e.getMessage());
+ } finally {
+ if (inProgress) {
+ remoteRegistry.removeFromJars(jarManager.getJarName());
+ }
+ jarManager.cleanUp();
+ }
+ }
+
+ /**
+ * Validates jar against remote jars to ensure no duplicate by jar name.
+ *
+ * @param remoteJars list of remote jars to validate against
+ * @param jarName jar name to be validated
+ * @throws JarValidationException in case of jar with the same name was found
+ */
+ private void validateAgainstRemoteRegistry(List remoteJars, String jarName) {
+ for (Jar remoteJar : remoteJars) {
+ if (remoteJar.getName().equals(jarName)) {
+ throw new JarValidationException(String.format("Jar with %s name has been already registered", jarName));
+ }
+ }
+ }
+
+ /**
+ * Instantiates remote registration. First gets remote schema registry with version.
+ * Version is used to ensure that we update the same registry we validated against.
+ * Then validates against list of remote jars.
+ * If validation is successful, first copies jar to registry area and starts updating remote schema registry.
+ * If during update {@link VersionMismatchException} was detected,
+ * attempts to repeat remote registration process till retry attempts exceeds the limit.
+ * If retry attempts number hits 0, throws exception that failed to update remote schema registry.
+ * In case of any error, if jar has been already copied to registry area, it will be deleted.
+ *
+ * @param jarManager helper class for copying jar to registry area
+ * @param remoteRegistry remote schema registry
+ * @throws IOException in case of problems with copying jar to registry area
+ */
+ private void initRemoteRegistration(JarManager jarManager,
+ RemoteDaffodilSchemaRegistry remoteRegistry) throws IOException {
+ int retryAttempts = remoteRegistry.getRetryAttempts();
+ boolean copyJar = true;
+ try {
+ while (retryAttempts >= 0) {
+ DataChangeVersion version = new DataChangeVersion();
+ List remoteJars = remoteRegistry.getRegistry(version).getJarList();
+ validateAgainstRemoteRegistry(remoteJars, jarManager.getJarName());
+ if (copyJar) {
+ jarManager.copyToRegistryArea();
+ copyJar = false;
+ }
+ List jars = Lists.newArrayList(remoteJars);
+ jars.add(Jar.newBuilder().setName(jarManager.getJarName()).build());
+ Registry updatedRegistry = Registry.newBuilder().addAllJar(jars).build();
+ try {
+ remoteRegistry.updateRegistry(updatedRegistry, version);
+ return;
+ } catch (VersionMismatchException ex) {
+ logger.debug("Failed to update schema registry during registration, version mismatch was detected.", ex);
+ retryAttempts--;
+ }
+ }
+ throw new DrillRuntimeException("Failed to update remote schema registry. Exceeded retry attempts limit.");
+ } catch (Exception e) {
+ if (!copyJar) {
+ jarManager.deleteQuietlyFromRegistryArea();
+ }
+ throw e;
+ }
+ }
+
+ /**
+ * Inner helper class that encapsulates logic for working with schema jars.
+ * During initialization it creates path to staging jar, remote temporary jar, and registry jar.
+ * Is responsible for validation, copying and deletion actions.
+ */
+ private class JarManager {
+
+ private final String jarName;
+ private final FileSystem fs;
+
+ private final Path remoteTmpDir;
+
+ private final Path stagingJar;
+ private final Path tmpRemoteJar;
+ private final Path registryJar;
+
+ JarManager(SqlNode sqlNode, RemoteDaffodilSchemaRegistry remoteRegistry) throws ForemanSetupException {
+ SqlCreateDaffodilSchema node = unwrap(sqlNode, SqlCreateDaffodilSchema.class);
+ this.jarName = ((SqlCharStringLiteral) node.getJar()).toValue();
+
+ this.stagingJar = new Path(remoteRegistry.getStagingArea(), jarName);
+
+ this.remoteTmpDir = new Path(remoteRegistry.getTmpArea(), UUID.randomUUID().toString());
+ this.tmpRemoteJar = new Path(remoteTmpDir, jarName);
+
+ this.registryJar = new Path(remoteRegistry.getRegistryArea(), jarName);
+
+ this.fs = remoteRegistry.getFs();
+ }
+
+ /**
+ * @return jar name
+ */
+ String getJarName() {
+ return jarName;
+ }
+
+ /**
+ * Validates that schema jar is present in staging area.
+ * Backs up jar to unique folder in remote temporary area.
+ *
+ * @throws IOException in case of jar absence or problems during copying jar
+ */
+ void initRemoteBackup() throws IOException {
+ checkPathExistence(stagingJar);
+ fs.mkdirs(remoteTmpDir);
+ FileUtil.copy(fs, stagingJar, fs, tmpRemoteJar, false, true, fs.getConf());
+ }
+
+ /**
+ * Copies schema jar to registry area.
+ *
+ * @throws IOException is re-thrown in case of problems during copying process
+ */
+ void copyToRegistryArea() throws IOException {
+ FileUtil.copy(fs, tmpRemoteJar, fs, registryJar, false, true, fs.getConf());
+ }
+
+ /**
+ * Deletes schema jar from staging area, in case of problems, logs warning and proceeds.
+ */
+ void deleteQuietlyFromStagingArea() {
+ deleteQuietly(stagingJar, false);
+ }
+
+ /**
+ * Deletes schema jar from registry area, in case of problems, logs warning and proceeds.
+ */
+ void deleteQuietlyFromRegistryArea() {
+ deleteQuietly(registryJar, false);
+ }
+
+ /**
+ * Removes quietly remote temporary folder.
+ */
+ void cleanUp() {
+ deleteQuietly(remoteTmpDir, true);
+ }
+ /**
+ * Checks if passed path exists on predefined file system.
+ *
+ * @param path path to be checked
+ * @throws IOException if path does not exist
+ */
+ private void checkPathExistence(Path path) throws IOException {
+ if (!fs.exists(path)) {
+ throw new IOException(String.format("File %s does not exist on file system %s",
+ path.toUri().getPath(), fs.getUri()));
+ }
+ }
+
+ /**
+ * Deletes quietly file or directory, in case of errors, logs warning and proceeds.
+ *
+ * @param path path to file or directory
+ * @param isDirectory set to true if we need to delete a directory
+ */
+ private void deleteQuietly(Path path, boolean isDirectory) {
+ try {
+ fs.delete(path, isDirectory);
+ } catch (IOException e) {
+ logger.warn(String.format("Error during deletion [%s]", path.toUri().getPath()), e);
+ }
+ }
+ }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/CreateFunctionHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/CreateFunctionHandler.java
index 0c9fd7905f0..578a400d7f0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/CreateFunctionHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/CreateFunctionHandler.java
@@ -314,7 +314,6 @@ void cleanUp() {
FileUtils.deleteQuietly(new File(localTmpDir.toUri()));
deleteQuietly(remoteTmpDir, true);
}
-
/**
* Checks if passed path exists on predefined file system.
*
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DropDaffodilSchemaHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DropDaffodilSchemaHandler.java
new file mode 100644
index 00000000000..c12de7a7518
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DropDaffodilSchemaHandler.java
@@ -0,0 +1,152 @@
+/*
+ * 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.planner.sql.handlers;
+
+import com.google.common.collect.Lists;
+import org.apache.calcite.sql.SqlCharStringLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.drill.common.exceptions.DrillRuntimeException;
+import org.apache.drill.exec.exception.VersionMismatchException;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.planner.sql.DirectPlan;
+import org.apache.drill.exec.planner.sql.parser.SqlDropDaffodilSchema;
+import org.apache.drill.exec.proto.UserBitShared.Jar;
+import org.apache.drill.exec.proto.UserBitShared.Registry;
+import org.apache.drill.exec.schema.daffodil.RemoteDaffodilSchemaRegistry;
+import org.apache.drill.exec.store.sys.store.DataChangeVersion;
+import org.apache.drill.exec.work.foreman.ForemanSetupException;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+public class DropDaffodilSchemaHandler extends DefaultSqlHandler {
+ private static Logger logger = LoggerFactory.getLogger(DropDaffodilSchemaHandler.class);
+
+ public DropDaffodilSchemaHandler(SqlHandlerConfig config) {
+ super(config);
+ }
+
+ /**
+ * Unregisters Daffodil schema JARs dynamically. Process consists of several steps:
+ *
+ * - Registering jar in jar registry to ensure that the jar is not being unregistered elsewhere.
+ * - Starts remote unregistration process, gets list of all jars and excludes jar to be deleted.
+ * - Removes jar from registry area.
+ *
+ *
+ * Only jars registered dynamically can be unregistered.
+ *
+ * Limitation: before jar unregistration make sure no one is using schemas from this jar.
+ * There is no guarantee that running queries will finish successfully or give correct result.
+ *
+ * @return - Single row indicating successful unregistration, raise exception otherwise
+ */
+ @Override
+ public PhysicalPlan getPlan(SqlNode sqlNode) throws ForemanSetupException, IOException {
+ SqlDropDaffodilSchema node = unwrap(sqlNode, SqlDropDaffodilSchema.class);
+ String jarName = ((SqlCharStringLiteral) node.getJar()).toValue();
+ RemoteDaffodilSchemaRegistry remoteSchemaRegistry = context.getDaffodilSchemaRegistry();
+
+ boolean inProgress = false;
+ try {
+ final String action = remoteSchemaRegistry.addToJars(jarName, RemoteDaffodilSchemaRegistry.Action.UNREGISTRATION);
+ if (!(inProgress = action == null)) {
+ return DirectPlan.createDirectPlan(context, false, String.format("Jar with %s name is used. Action: %s", jarName, action));
+ }
+
+ Jar deletedJar = unregister(jarName, remoteSchemaRegistry);
+ if (deletedJar == null) {
+ return DirectPlan.createDirectPlan(context, false, String.format("Jar %s is not registered in remote registry", jarName));
+ }
+
+ removeJarFromArea(jarName, remoteSchemaRegistry.getFs(), remoteSchemaRegistry.getRegistryArea());
+
+ return DirectPlan.createDirectPlan(context, true,
+ String.format("Daffodil schema jar %s has been unregistered successfully.", jarName));
+
+ } catch (Exception e) {
+ logger.error("Error during Daffodil schema unregistration", e);
+ return DirectPlan.createDirectPlan(context, false, e.getMessage());
+ } finally {
+ if (inProgress) {
+ remoteSchemaRegistry.removeFromJars(jarName);
+ }
+ }
+ }
+
+ /**
+ * Gets remote schema registry with version.
+ * Version is used to ensure that we update the same registry we removed jars from.
+ * Looks for a jar to be deleted, if found,
+ * attempts to update remote registry with list of jars, that excludes jar to be deleted.
+ * If during update {@link VersionMismatchException} was detected,
+ * attempts to repeat unregistration process till retry attempts exceeds the limit.
+ * If retry attempts number hits 0, throws exception that failed to update remote schema registry.
+ *
+ * @param jarName jar name
+ * @param remoteSchemaRegistry remote schema registry
+ * @return jar that was unregistered, null otherwise
+ */
+ private Jar unregister(String jarName, RemoteDaffodilSchemaRegistry remoteSchemaRegistry) {
+ int retryAttempts = remoteSchemaRegistry.getRetryAttempts();
+ while (retryAttempts >= 0) {
+ DataChangeVersion version = new DataChangeVersion();
+ Registry registry = remoteSchemaRegistry.getRegistry(version);
+ Jar jarToBeDeleted = null;
+ List jars = Lists.newArrayList();
+ for (Jar j : registry.getJarList()) {
+ if (j.getName().equals(jarName)) {
+ jarToBeDeleted = j;
+ } else {
+ jars.add(j);
+ }
+ }
+ if (jarToBeDeleted == null) {
+ return null;
+ }
+ Registry updatedRegistry = Registry.newBuilder().addAllJar(jars).build();
+ try {
+ remoteSchemaRegistry.updateRegistry(updatedRegistry, version);
+ return jarToBeDeleted;
+ } catch (VersionMismatchException ex) {
+ logger.debug("Failed to update schema registry during unregistration, version mismatch was detected.", ex);
+ retryAttempts--;
+ }
+ }
+ throw new DrillRuntimeException("Failed to update remote schema registry. Exceeded retry attempts limit.");
+ }
+
+ /**
+ * Removes jar from indicated area, in case of error log it and proceeds.
+ *
+ * @param jarName jar name
+ * @param fs file system
+ * @param area path to area
+ */
+ private void removeJarFromArea(String jarName, FileSystem fs, Path area) {
+ try {
+ fs.delete(new Path(area, jarName), false);
+ } catch (IOException e) {
+ logger.error("Error removing jar {} from area {}", jarName, area.toUri().getPath());
+ }
+ }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlCreateDaffodilSchema.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlCreateDaffodilSchema.java
new file mode 100644
index 00000000000..29a1c092f14
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlCreateDaffodilSchema.java
@@ -0,0 +1,80 @@
+/*
+ * 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.planner.sql.parser;
+
+import com.google.common.collect.Lists;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.drill.exec.planner.sql.handlers.AbstractSqlHandler;
+import org.apache.drill.exec.planner.sql.handlers.CreateDaffodilSchemaHandler;
+import org.apache.drill.exec.planner.sql.handlers.SqlHandlerConfig;
+
+import java.util.List;
+
+public class SqlCreateDaffodilSchema extends DrillSqlCall {
+
+ private final SqlNode jar;
+
+ public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("CREATE_DAFFODIL_SCHEMA", SqlKind.OTHER_DDL) {
+ @Override
+ public SqlCall createCall(SqlLiteral functionQualifier, SqlParserPos pos, SqlNode... operands) {
+ return new SqlCreateDaffodilSchema(pos, operands[0]);
+ }
+ };
+
+ public SqlCreateDaffodilSchema(SqlParserPos pos, SqlNode jar) {
+ super(pos);
+ this.jar = jar;
+ }
+
+ @Override
+ public SqlOperator getOperator() {
+ return OPERATOR;
+ }
+
+ @Override
+ public List getOperandList() {
+ List opList = Lists.newArrayList();
+ opList.add(jar);
+ return opList;
+ }
+
+ @Override
+ public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
+ writer.keyword("CREATE");
+ writer.keyword("DAFFODIL");
+ writer.keyword("SCHEMA");
+ writer.keyword("USING");
+ writer.keyword("JAR");
+ jar.unparse(writer, leftPrec, rightPrec);
+ }
+
+ @Override
+ public AbstractSqlHandler getSqlHandler(SqlHandlerConfig config) {
+ return new CreateDaffodilSchemaHandler(config);
+ }
+
+ public SqlNode getJar() { return jar; }
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropDaffodilSchema.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropDaffodilSchema.java
new file mode 100644
index 00000000000..8050e67d758
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropDaffodilSchema.java
@@ -0,0 +1,80 @@
+/*
+ * 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.planner.sql.parser;
+
+import com.google.common.collect.Lists;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.drill.exec.planner.sql.handlers.AbstractSqlHandler;
+import org.apache.drill.exec.planner.sql.handlers.DropDaffodilSchemaHandler;
+import org.apache.drill.exec.planner.sql.handlers.SqlHandlerConfig;
+
+import java.util.List;
+
+public class SqlDropDaffodilSchema extends DrillSqlCall {
+
+ private final SqlNode jar;
+
+ public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("DROP_DAFFODIL_SCHEMA", SqlKind.OTHER_DDL) {
+ @Override
+ public SqlCall createCall(SqlLiteral functionQualifier, SqlParserPos pos, SqlNode... operands) {
+ return new SqlDropDaffodilSchema(pos, operands[0]);
+ }
+ };
+
+ public SqlDropDaffodilSchema(SqlParserPos pos, SqlNode jar) {
+ super(pos);
+ this.jar = jar;
+ }
+
+ @Override
+ public SqlOperator getOperator() {
+ return OPERATOR;
+ }
+
+ @Override
+ public List getOperandList() {
+ List opList = Lists.newArrayList();
+ opList.add(jar);
+ return opList;
+ }
+
+ @Override
+ public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
+ writer.keyword("DROP");
+ writer.keyword("DAFFODIL");
+ writer.keyword("SCHEMA");
+ writer.keyword("USING");
+ writer.keyword("JAR");
+ jar.unparse(writer, leftPrec, rightPrec);
+ }
+
+ @Override
+ public AbstractSqlHandler getSqlHandler(SqlHandlerConfig config) {
+ return new DropDaffodilSchemaHandler(config);
+ }
+
+ public SqlNode getJar() { return jar; }
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/schema/daffodil/DaffodilSchemaProvider.java b/exec/java-exec/src/main/java/org/apache/drill/exec/schema/daffodil/DaffodilSchemaProvider.java
new file mode 100644
index 00000000000..ab230562f0f
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/schema/daffodil/DaffodilSchemaProvider.java
@@ -0,0 +1,55 @@
+/*
+ * 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.schema.daffodil;
+
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.scanner.persistence.ScanResult;
+import org.apache.drill.exec.coord.ClusterCoordinator;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.sys.PersistentStoreProvider;
+
+/**
+ * Class for managing Daffodil schemata. Schemata will be obtained via CREATE DAFFODIL SCHEMA queries.
+ */
+public class DaffodilSchemaProvider implements AutoCloseable {
+
+ private RemoteDaffodilSchemaRegistry remoteDaffodilSchemaRegistry;
+
+ public DaffodilSchemaProvider(DrillbitContext context) {
+ this(context.getConfig(), context.getStoreProvider(), context.getClusterCoordinator());
+ }
+
+ public DaffodilSchemaProvider(DrillConfig config, ScanResult classpathScan) {
+ // This constructor is incomplete - needs StoreProvider and ClusterCoordinator
+ }
+
+ public DaffodilSchemaProvider(DrillConfig config, PersistentStoreProvider storeProvider, ClusterCoordinator coordinator) {
+ this.remoteDaffodilSchemaRegistry = new RemoteDaffodilSchemaRegistry();
+ this.remoteDaffodilSchemaRegistry.init(config, storeProvider, coordinator);
+ }
+
+ public RemoteDaffodilSchemaRegistry getRemoteDaffodilSchemaRegistry() {
+ return remoteDaffodilSchemaRegistry;
+ }
+
+ @Override
+ public void close() throws Exception {
+ AutoCloseables.closeSilently(remoteDaffodilSchemaRegistry);
+ }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/schema/daffodil/DaffodilSchemaRegistry.java b/exec/java-exec/src/main/java/org/apache/drill/exec/schema/daffodil/DaffodilSchemaRegistry.java
new file mode 100644
index 00000000000..85ee369f6b9
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/schema/daffodil/DaffodilSchemaRegistry.java
@@ -0,0 +1,49 @@
+/*
+ * 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.schema.daffodil;
+
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+
+import java.util.Iterator;
+import java.util.Map;
+
+/**
+ * Persistent Registry for Daffodil Schemata
+ */
+public interface DaffodilSchemaRegistry extends AutoCloseable {
+
+ /**
+ * Creates a token table for specified {@code pluginName}.
+ * @param schemaName The name of the plugin instance.
+ */
+ void createSchemaTable(String schemaName);
+
+ PersistentSchemaTable getSchemaTable(String name);
+
+ /**
+ * Deletes the schema table for specified {@code pluginName}.
+ * @param pluginName name of the plugin whose token table should be removed
+ */
+ void deleteSchemaTable(String pluginName);
+
+ /**
+ * Returns iterator for aliases table entries.
+ */
+ Iterator> getAllSchemata();
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/schema/daffodil/DaffodilSchemata.java b/exec/java-exec/src/main/java/org/apache/drill/exec/schema/daffodil/DaffodilSchemata.java
new file mode 100644
index 00000000000..5b561e45a92
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/schema/daffodil/DaffodilSchemata.java
@@ -0,0 +1,69 @@
+/*
+ * 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.schema.daffodil;
+
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+
+public interface DaffodilSchemata {
+ /**
+ * Key of {@link this} Daffodil schema.
+ */
+ String getKey();
+
+ /**
+ * Gets the current Drill representation of a Daffodil schema
+ *
+ * @return The current access token
+ */
+ TupleMetadata getSchema();
+
+ /**
+ * Sets the schema
+ *
+ * @param drillSchema The incoming schema.
+ */
+ void setSchema(TupleMetadata drillSchema);
+
+ /**
+ * Returns value from schemata table that corresponds to provided plugin.
+ *
+ * @param schemaName name of desired Daffodil schema
+ * @return The Desired Daffodil schema or {@code null} if no such schema exists.
+ */
+ TupleMetadata get(String schemaName);
+
+ /**
+ * Associates provided token with provided plugin in token table.
+ *
+ * @param token Token of the value to associate with
+ * @param value Value that will be associated with provided alias
+ * @param replace Whether existing value for the same token should be replaced
+ * @return {@code true} if provided token was associated with
+ * the provided value in tokens table
+ */
+ boolean put(String token, String value, boolean replace);
+
+ /**
+ * Removes value for specified token from tokens table.
+ * @param token token of the value to remove
+ * @return {@code true} if the value associated with
+ * provided token was removed from the tokens table.
+ */
+ boolean remove(String schema);
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/schema/daffodil/PersistentSchemaRegistry.java b/exec/java-exec/src/main/java/org/apache/drill/exec/schema/daffodil/PersistentSchemaRegistry.java
new file mode 100644
index 00000000000..fc278ff35a0
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/schema/daffodil/PersistentSchemaRegistry.java
@@ -0,0 +1,119 @@
+/*
+ * 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.schema.daffodil;
+
+
+import com.fasterxml.jackson.databind.InjectableValues;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.drill.common.exceptions.DrillRuntimeException;
+import org.apache.drill.exec.exception.StoreException;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.sys.PersistentStore;
+import org.apache.drill.exec.store.sys.PersistentStoreConfig;
+
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.function.Supplier;
+
+/**
+ * Implementation of {@link DaffodilSchemaRegistry} that persists Daffodil schema tables
+ * to the preconfigured persistent store.
+ */
+
+// TODO Start here.. Add constructor and initialization
+
+public class PersistentSchemaRegistry implements DaffodilSchemaRegistry {
+ private final PersistentStore store;
+
+ public PersistentSchemaRegistry(DrillbitContext context, String registryPath) {
+ try {
+ ObjectMapper mapper = context.getLpPersistence().getMapper().copy();
+ InjectableValues injectables = new InjectableValues.Std()
+ .addValue(StoreProvider.class, new StoreProvider(this::getStore));
+
+ mapper.setInjectableValues(injectables);
+ this.store = context
+ .getStoreProvider()
+ .getOrCreateStore(PersistentStoreConfig
+ .newJacksonBuilder(mapper, PersistentSchemaTable.class)
+ .name(registryPath)
+ .build());
+ } catch (StoreException e) {
+ throw new DrillRuntimeException(
+ "Failure while reading and loading Daffodil schema table.");
+ }
+ }
+
+ public PersistentStore getStore() {
+ return store;
+ }
+
+ @Override
+ public PersistentSchemaTable getSchemaTable(String name) {
+ name = name.toLowerCase();
+ if (!store.contains(name)) {
+ createSchemaTable(name);
+ }
+ return store.get(name);
+ }
+
+ @SuppressWarnings({"unchecked", "rawtypes"})
+ public Iterator> getAllSchemata() {
+ return (Iterator) store.getAll();
+ }
+
+ @Override
+ public void createSchemaTable(String pluginName) {
+ // In Drill, Storage plugin names are stored in lower case. These checks make sure
+ // that the tokens are associated with the correct plugin
+ pluginName = pluginName.toLowerCase();
+ if (!store.contains(pluginName)) {
+ PersistentSchemaTable schemaTable =
+ new PersistentSchemaTable(new HashMap<>(), pluginName, new StoreProvider(this::getStore));
+ store.put(pluginName, schemaTable);
+ }
+ }
+
+ @Override
+ public void deleteSchemaTable(String pluginName) {
+ pluginName = pluginName.toLowerCase();
+ if (store.contains(pluginName)) {
+ store.delete(pluginName);
+ }
+ }
+
+ @Override
+ public void close() throws Exception {
+ store.close();
+ }
+
+ public static class StoreProvider {
+ private final Supplier> supplier;
+
+ public StoreProvider(Supplier> supplier) {
+ this.supplier = supplier;
+ }
+
+ public PersistentStore getStore() {
+ return supplier.get();
+ }
+ }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/schema/daffodil/PersistentSchemaTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/schema/daffodil/PersistentSchemaTable.java
new file mode 100644
index 00000000000..331aafca995
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/schema/daffodil/PersistentSchemaTable.java
@@ -0,0 +1,96 @@
+/*
+ * 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.schema.daffodil;
+
+
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.store.sys.PersistentStore;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Implementation of Daffodil schema table that updates its version in persistent store after modifications.
+ */
+public class PersistentSchemaTable implements DaffodilSchemata {
+ public final String SCHEMA_KEY = "schema";
+ private final Map schemata;
+ private final String key;
+ private final PersistentStore store;
+
+ @JsonCreator
+ public PersistentSchemaTable(
+ @JsonProperty("schemata") Map schemata,
+ @JsonProperty("key") String key,
+ @JacksonInject PersistentSchemaRegistry.StoreProvider storeProvider) {
+ this.schemata = schemata != null ? schemata : new HashMap<>();
+ this.key = key;
+ this.store = storeProvider.getStore();
+ }
+
+ @Override
+ @JsonProperty("key")
+ public String getKey() {
+ return key;
+ }
+
+ @Override
+ public TupleMetadata getSchema() {
+ return null;
+ }
+
+ @Override
+ public void setSchema(TupleMetadata drillSchema) {
+
+ }
+
+ @Override
+ @JsonIgnore
+ public TupleMetadata get(String schemaName) {
+ return schemata.get(schemaName);
+ }
+
+ @Override
+ @JsonIgnore
+ public boolean put(String token, String value, boolean replace) {
+ if (replace || ! schemata.containsKey(token)) {
+ schemata.put(token, TupleMetadata.of(value));
+ store.put(key, this);
+ return true;
+ }
+ return false;
+ }
+
+ @Override
+ @JsonIgnore
+ public boolean remove(String schemaName) {
+ boolean isRemoved = schemata.remove(schemaName) != null;
+ store.put(key, this);
+ return isRemoved;
+ }
+
+ @JsonProperty("schemata")
+ public Map getSchemata() {
+ return schemata;
+ }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/schema/daffodil/README.md b/exec/java-exec/src/main/java/org/apache/drill/exec/schema/daffodil/README.md
new file mode 100644
index 00000000000..2459bbad8c4
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/schema/daffodil/README.md
@@ -0,0 +1,202 @@
+# Daffodil Schema Management in Apache Drill
+
+This package implements dynamic Daffodil schema management for Apache Drill, allowing users to register and manage Daffodil (DFDL) schema files at runtime using SQL commands.
+
+## Overview
+
+The Daffodil schema management system allows users to:
+- Register Daffodil schema JARs dynamically without restarting Drill
+- Distribute schema JARs across all Drillbits in a cluster
+- Unregister schemas when no longer needed
+- Version control schema registrations
+
+## Architecture
+
+### Key Components
+
+1. **RemoteDaffodilSchemaRegistry**: Main registry managing schema JARs across the cluster
+ - Persistent storage for registered schemas
+ - Transient locks for concurrent access control
+ - Version-controlled updates with retry logic
+
+2. **DaffodilSchemaProvider**: Lifecycle manager for schema registry
+ - Initialized during Drillbit startup
+ - Provides access to RemoteDaffodilSchemaRegistry
+
+3. **CreateDaffodilSchemaHandler**: Handles schema registration SQL commands
+4. **DropDaffodilSchemaHandler**: Handles schema unregistration SQL commands
+
+### File System Layout
+
+The system uses three directories for managing schema JARs:
+
+- **Staging**: `/path/to/base/staging/`
+ - Users copy schema JAR files here before registration
+ - Temporary location before validation and registration
+
+- **Registry**: `/path/to/base/registry/`
+ - Permanent storage for registered schema JARs
+ - Accessible by all Drillbits in the cluster
+
+- **Tmp**: `/path/to/base/tmp/`
+ - Temporary backup during registration process
+ - Used for rollback on registration failure
+
+## Configuration
+
+Configure Daffodil schema directories in `drill-override.conf`:
+
+```hocon
+drill.exec.daffodil {
+ directory {
+ # Base directory for all Daffodil schema directories
+ base: "/opt/drill/daffodil"
+
+ # Optional: specific filesystem (defaults to default FS)
+ # fs: "hdfs://namenode:8020"
+
+ # Optional: root directory (defaults to user home)
+ # root: "/user/drill"
+
+ # Staging directory for uploading schema JARs
+ staging: ${drill.exec.daffodil.directory.base}"/staging"
+
+ # Registry directory for registered schemas
+ registry: ${drill.exec.daffodil.directory.base}"/registry"
+
+ # Temporary directory for backup during registration
+ tmp: ${drill.exec.daffodil.directory.base}"/tmp"
+ }
+}
+```
+
+## Usage
+
+### Registering a Daffodil Schema
+
+1. **Copy the schema JAR to the staging directory:**
+
+```bash
+# Copy your Daffodil schema JAR to the configured staging directory
+cp my-schema.jar /opt/drill/daffodil/staging/
+```
+
+2. **Register the schema using SQL:**
+
+```sql
+CREATE DAFFODIL SCHEMA USING JAR 'my-schema.jar';
+```
+
+**Success Response:**
+```
++------+----------------------------------------------------------+
+| ok | summary |
++------+----------------------------------------------------------+
+| true | Daffodil schema jar my-schema.jar has been registered |
+| | successfully. |
++------+----------------------------------------------------------+
+```
+
+### Unregistering a Daffodil Schema
+
+```sql
+DROP DAFFODIL SCHEMA USING JAR 'my-schema.jar';
+```
+
+**Success Response:**
+```
++------+------------------------------------------------------------+
+| ok | summary |
++------+------------------------------------------------------------+
+| true | Daffodil schema jar my-schema.jar has been unregistered |
+| | successfully. |
++------+------------------------------------------------------------+
+```
+
+## Error Handling
+
+### Common Errors
+
+**JAR not found in staging:**
+```
+File /opt/drill/daffodil/staging/my-schema.jar does not exist on file system
+```
+**Solution:** Ensure the JAR file is copied to the staging directory
+
+**Duplicate schema registration:**
+```
+Jar with my-schema.jar name has been already registered
+```
+**Solution:** Use DROP to unregister the existing schema first, or use a different JAR name
+
+**Schema not registered:**
+```
+Jar my-schema.jar is not registered in remote registry
+```
+**Solution:** Verify the schema was previously registered using CREATE DAFFODIL SCHEMA
+
+**Concurrent access:**
+```
+Jar with my-schema.jar name is used. Action: REGISTRATION
+```
+**Solution:** Wait for the current operation to complete
+
+## Registration Process Flow
+
+1. **Validation**: Checks JAR exists in staging area
+2. **Locking**: Acquires lock to prevent concurrent operations
+3. **Backup**: Copies JAR to temporary area
+4. **Duplicate Check**: Validates no duplicate in remote registry
+5. **Registration**: Copies JAR to registry area
+6. **Update**: Updates persistent registry with version control
+7. **Cleanup**: Removes JAR from staging area
+8. **Unlock**: Releases lock
+
+## Unregistration Process Flow
+
+1. **Locking**: Acquires lock for unregistration
+2. **Lookup**: Finds JAR in remote registry
+3. **Update**: Removes from persistent registry
+4. **Deletion**: Deletes JAR from registry area
+5. **Unlock**: Releases lock
+
+## Best Practices
+
+1. **Naming Convention**: Use descriptive, versioned names for schema JARs
+ - Good: `customer-schema-v1.0.jar`
+ - Avoid: `schema.jar`
+
+2. **Testing**: Test schemas in a development environment before production
+
+3. **Backup**: Keep backup copies of schema JARs outside the Drill directories
+
+4. **Cleanup**: Remove unused schemas with DROP DAFFODIL SCHEMA
+
+5. **Concurrent Access**: Avoid registering/unregistering the same schema simultaneously from multiple clients
+
+## Development and Testing
+
+### Running Tests
+
+```bash
+# Run all Daffodil schema tests
+mvn test -pl exec/java-exec -Dtest=TestDaffodilSchemaHandlers
+
+# Run RemoteDaffodilSchemaRegistry tests
+mvn test -pl exec/java-exec -Dtest=TestRemoteDaffodilSchemaRegistry
+```
+
+### Test Coverage
+
+- Basic syntax validation
+- JAR registration and unregistration
+- Duplicate detection
+- Concurrent access handling
+- Error scenarios (missing JAR, not registered, etc.)
+- File system operations
+
+## Related Documentation
+
+- [Apache Daffodil](https://daffodil.apache.org/)
+- [DFDL Specification](https://www.ogf.org/ogf/doku.php/standards/dfdl/dfdl)
+- [Drill Dynamic UDFs](https://drill.apache.org/docs/dynamic-udfs/)
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/schema/daffodil/RemoteDaffodilSchemaRegistry.java b/exec/java-exec/src/main/java/org/apache/drill/exec/schema/daffodil/RemoteDaffodilSchemaRegistry.java
new file mode 100644
index 00000000000..9d6573c9452
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/schema/daffodil/RemoteDaffodilSchemaRegistry.java
@@ -0,0 +1,263 @@
+/*
+ * 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.schema.daffodil;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.drill.common.util.JacksonUtils;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.exceptions.DrillRuntimeException;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.coord.ClusterCoordinator;
+import org.apache.drill.exec.coord.store.TransientStore;
+import org.apache.drill.exec.coord.store.TransientStoreConfig;
+import org.apache.drill.exec.exception.StoreException;
+import org.apache.drill.exec.exception.VersionMismatchException;
+import org.apache.drill.exec.proto.SchemaUserBitShared;
+import org.apache.drill.exec.proto.UserBitShared.Registry;
+import org.apache.drill.exec.store.sys.PersistentStoreConfig;
+import org.apache.drill.exec.store.sys.PersistentStoreProvider;
+import org.apache.drill.exec.store.sys.VersionedPersistentStore;
+import org.apache.drill.exec.store.sys.store.DataChangeVersion;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static com.fasterxml.jackson.databind.SerializationFeature.INDENT_OUTPUT;
+
+/**
+ * Is responsible for remote Daffodil schema registry management.
+ * Creates all remote registry areas at startup and validates them.
+ *
+ * Similar to RemoteFunctionRegistry but for Daffodil schemas.
+ *
+ * There are two schema stores:
+ *
+ * - REGISTRY - persistent store, stores remote schema registry {@link Registry} under daffodil_schema path
+ * which contains information about all dynamically registered schema jars.
+ *
+ * - JARS - transient store, stores information under daffodil_schema/jars path.
+ * Serves as lock, not allowing to perform any action on the same jar at the same time.
+ * There are two types of actions: {@link Action#REGISTRATION} and {@link Action#UNREGISTRATION}.
+ *
+ * There are three schema areas:
+ *
+ * - STAGING - area where user copies schema jars before starting registration process.
+ * - REGISTRY - area where registered schema jars are stored.
+ * - TMP - area where schema jars are backed up in unique folder during registration process.
+ */
+public class RemoteDaffodilSchemaRegistry implements AutoCloseable {
+
+ private static final String REGISTRY_PATH = "registry";
+ private static final Logger logger = LoggerFactory.getLogger(RemoteDaffodilSchemaRegistry.class);
+ private static final ObjectMapper mapper = JacksonUtils.createObjectMapper().enable(INDENT_OUTPUT);
+
+ private int retryAttempts;
+ private FileSystem fs;
+ private Path registryArea;
+ private Path stagingArea;
+ private Path tmpArea;
+
+ private VersionedPersistentStore registry;
+ private TransientStore jars;
+
+ public RemoteDaffodilSchemaRegistry() {
+ }
+
+ public void init(DrillConfig config, PersistentStoreProvider storeProvider, ClusterCoordinator coordinator) {
+ prepareStores(storeProvider, coordinator);
+ prepareAreas(config);
+ this.retryAttempts = config.getInt(ExecConstants.UDF_RETRY_ATTEMPTS); // Reuse UDF retry attempts config
+ }
+
+ /**
+ * Returns current remote schema registry version.
+ * If remote schema registry is not found or unreachable, logs error and returns -1.
+ *
+ * @return remote schema registry version if any, -1 otherwise
+ */
+ public int getRegistryVersion() {
+ DataChangeVersion version = new DataChangeVersion();
+ boolean contains = false;
+ try {
+ contains = registry.contains(REGISTRY_PATH, version);
+ } catch (Exception e) {
+ logger.error("Problem during trying to access remote Daffodil schema registry [{}]", REGISTRY_PATH, e);
+ }
+ if (contains) {
+ return version.getVersion();
+ } else {
+ logger.error("Remote Daffodil schema registry [{}] is unreachable", REGISTRY_PATH);
+ return DataChangeVersion.NOT_AVAILABLE;
+ }
+ }
+
+ /**
+ * Report whether a remote registry exists.
+ * @return true if a remote registry exists, false otherwise
+ */
+ public boolean hasRegistry() {
+ return registry != null;
+ }
+
+ public Registry getRegistry(DataChangeVersion version) {
+ return registry.get(REGISTRY_PATH, version);
+ }
+
+ public void updateRegistry(Registry registryContent, DataChangeVersion version) throws VersionMismatchException {
+ registry.put(REGISTRY_PATH, registryContent, version);
+ }
+
+ public String addToJars(String jar, Action action) {
+ return jars.putIfAbsent(jar, action.toString());
+ }
+
+ public void removeFromJars(String jar) {
+ jars.remove(jar);
+ }
+
+ public int getRetryAttempts() {
+ return retryAttempts;
+ }
+
+ public FileSystem getFs() {
+ return fs;
+ }
+
+ public Path getRegistryArea() {
+ return registryArea;
+ }
+
+ public Path getStagingArea() {
+ return stagingArea;
+ }
+
+ public Path getTmpArea() {
+ return tmpArea;
+ }
+
+ /**
+ * Connects to two stores: REGISTRY and JARS.
+ * Puts in REGISTRY store with default instance of remote schema registry if store is initiated for the first time.
+ */
+ private void prepareStores(PersistentStoreProvider storeProvider, ClusterCoordinator coordinator) {
+ try {
+ PersistentStoreConfig registrationConfig = PersistentStoreConfig
+ .newProtoBuilder(SchemaUserBitShared.Registry.WRITE, SchemaUserBitShared.Registry.MERGE)
+ .name("daffodil_schema")
+ .persist()
+ .build();
+ registry = storeProvider.getOrCreateVersionedStore(registrationConfig);
+ logger.trace("Remote Daffodil schema registry type: {}.", registry.getClass());
+ registry.putIfAbsent(REGISTRY_PATH, Registry.getDefaultInstance());
+ } catch (StoreException e) {
+ throw new DrillRuntimeException("Failure while loading remote Daffodil schema registry.", e);
+ }
+
+ TransientStoreConfig jarsConfig = TransientStoreConfig.
+ newJacksonBuilder(mapper, String.class).name("daffodil_schema/jars").build();
+ jars = coordinator.getOrCreateTransientStore(jarsConfig);
+ }
+
+ /**
+ * Creates if absent and validates three Daffodil schema areas: STAGING, REGISTRY and TMP.
+ * Generated schema areas root from {@link ExecConstants#DFDL_DIRECTORY_ROOT},
+ * if not set, uses user home directory instead.
+ */
+ private void prepareAreas(DrillConfig config) {
+ logger.info("Preparing three remote Daffodil schema areas: staging, registry and tmp.");
+ Configuration conf = new Configuration();
+ if (config.hasPath(ExecConstants.DFDL_DIRECTORY_FS)) {
+ conf.set(FileSystem.FS_DEFAULT_NAME_KEY, config.getString(ExecConstants.DFDL_DIRECTORY_FS));
+ }
+
+ try {
+ this.fs = FileSystem.get(conf);
+ } catch (IOException e) {
+ throw DrillRuntimeException.create(e,
+ "Error during file system %s setup", conf.get(FileSystem.FS_DEFAULT_NAME_KEY));
+ }
+
+ String root = fs.getHomeDirectory().toUri().getPath();
+ if (config.hasPath(ExecConstants.DFDL_DIRECTORY_ROOT)) {
+ root = config.getString(ExecConstants.DFDL_DIRECTORY_ROOT);
+ }
+
+ this.registryArea = createArea(fs, root, config.getString(ExecConstants.DFDL_DIRECTORY_REGISTRY));
+ this.stagingArea = createArea(fs, root, config.getString(ExecConstants.DFDL_DIRECTORY_STAGING));
+ this.tmpArea = createArea(fs, root, config.getString(ExecConstants.DFDL_DIRECTORY_TMP));
+ }
+
+ /**
+ * Concatenates schema area with root directory.
+ * Creates schema area, if area does not exist.
+ * Checks if area exists and is directory, if it is writable for current user,
+ * throws {@link org.apache.drill.common.exceptions.DrillRuntimeException} otherwise.
+ *
+ * @param fs file system where area should be created
+ * @param root root directory
+ * @param directory directory path
+ * @return path to area
+ */
+ private Path createArea(FileSystem fs, String root, String directory) {
+ Path path = new Path(root, directory);
+ String filePath = path.toUri().getPath();
+ try {
+ if (!fs.exists(path)) {
+ fs.mkdirs(path, new FsPermission(FsAction.ALL, FsAction.READ_EXECUTE, FsAction.NONE));
+ logger.info("Created Daffodil schema directory at [{}].", filePath);
+ }
+
+ FileStatus fileStatus = fs.getFileStatus(path);
+ if (!fileStatus.isDirectory()) {
+ throw new DrillRuntimeException(String.format("Indicated path [%s] is not a directory.", filePath));
+ }
+
+ FsPermission permission = fileStatus.getPermission();
+ FsAction userAction = permission.getUserAction();
+ if (!userAction.implies(FsAction.READ_WRITE)) {
+ throw new DrillRuntimeException(String.format("Unable to read or write into Daffodil schema directory [%s].", filePath));
+ }
+
+ logger.info("Daffodil schema directory [{}] has been validated.", filePath);
+ return path;
+ } catch (IOException e) {
+ throw DrillRuntimeException.create(e, "Error during Daffodil schema area creation [%s].", filePath);
+ }
+ }
+
+ @Override
+ public void close() throws Exception {
+ AutoCloseables.closeSilently(registry, jars);
+ }
+
+ /**
+ * Enum for jar actions.
+ */
+ public enum Action {
+ REGISTRATION,
+ UNREGISTRATION
+ }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
index 7daa5d26959..3fa9bcede8c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
@@ -42,6 +42,8 @@
import org.apache.drill.exec.rpc.user.UserServer;
import org.apache.drill.exec.rpc.user.UserServer.BitToUserConnection;
import org.apache.drill.exec.rpc.user.UserServer.BitToUserConnectionConfig;
+import org.apache.drill.exec.schema.daffodil.DaffodilSchemaProvider;
+import org.apache.drill.exec.schema.daffodil.RemoteDaffodilSchemaRegistry;
import org.apache.drill.exec.server.options.SystemOptionManager;
import org.apache.drill.exec.store.SchemaFactory;
import org.apache.drill.exec.store.StoragePluginRegistry;
@@ -82,6 +84,7 @@ public class DrillbitContext implements AutoCloseable {
private ResourceManager resourceManager;
private final MetastoreRegistry metastoreRegistry;
private final DrillCounters counters;
+ private final DaffodilSchemaProvider daffodilSchemaProvider;
public DrillbitContext(
DrillbitEndpoint endpoint,
@@ -133,6 +136,9 @@ public DrillbitContext(
this.aliasRegistryProvider = new AliasRegistryProvider(this);
this.oAuthTokenProvider = new OAuthTokenProvider(this);
+ // TODO Start here and complete the initialization process
+ this.daffodilSchemaProvider = new DaffodilSchemaProvider(config, provider, coord);
+
this.counters = DrillCounters.getInstance();
}
@@ -284,6 +290,14 @@ public RemoteFunctionRegistry getRemoteFunctionRegistry() {
return functionRegistry.getRemoteFunctionRegistry();
}
+ public DaffodilSchemaProvider getDaffodilSchemaProvider() {
+ return daffodilSchemaProvider;
+ }
+
+ public RemoteDaffodilSchemaRegistry getRemoteDaffodilSchemaRegistry() {
+ return daffodilSchemaProvider.getRemoteDaffodilSchemaRegistry();
+ }
+
/**
* Use the operator table built during startup when "exec.udf.use_dynamic" option
* is set to false.
@@ -318,6 +332,7 @@ public void close() throws Exception {
getMetastoreRegistry().close();
getAliasRegistryProvider().close();
getOauthTokenProvider().close();
+ getDaffodilSchemaProvider().close();
}
public ResourceManager getResourceManager() {
diff --git a/exec/java-exec/src/main/resources/drill-module.conf b/exec/java-exec/src/main/resources/drill-module.conf
index 7541a99e2dd..c06eb6334d8 100644
--- a/exec/java-exec/src/main/resources/drill-module.conf
+++ b/exec/java-exec/src/main/resources/drill-module.conf
@@ -410,6 +410,31 @@ drill.exec: {
threadpool_size: 8,
decode_threadpool_size: 1
},
+ daffodil: {
+ retry-attempts: 5,
+ directory: {
+ # Base directory for remote and local daffodil directories, unique among clusters.
+ base: ${drill.exec.zk.root}"/daffodil",
+
+ # Path to local daffodil, always created on local file system.
+ # Root for these directory is generated at runtime unless Drill temporary directory is set.
+ local: ${drill.exec.daffodil.directory.base}"/local",
+
+ # Set this property if custom file system should be used to create remote directories, ex: fs: "file:///".
+ # fs: "",
+ # Set this property if custom absolute root should be used for remote directories, ex: root: "/app/drill".
+ # root: "",
+
+ # Relative path to all remote daffodil directories.
+ # Directories are created under default file system taken from Hadoop configuration
+ # unless ${drill.exec.daffodil.directory.fs} is set.
+ # User home directory is used as root unless ${drill.exec.daffodil.directory.root} is set.
+ staging: ${drill.exec.daffodil.directory.base}"/staging",
+ registry: ${drill.exec.daffodil.directory.base}"/registry",
+ tmp: ${drill.exec.daffodil.directory.base}"/tmp"
+ }
+ },
+
udf: {
retry-attempts: 5,
# Disables (parts of) the dynamic UDF functionality.
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/planner/sql/handlers/TestDaffodilSchemaHandlers.java b/exec/java-exec/src/test/java/org/apache/drill/exec/planner/sql/handlers/TestDaffodilSchemaHandlers.java
new file mode 100644
index 00000000000..2aaf1d467e7
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/planner/sql/handlers/TestDaffodilSchemaHandlers.java
@@ -0,0 +1,244 @@
+/*
+ * 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.planner.sql.handlers;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.drill.categories.SlowTest;
+import org.apache.drill.categories.SqlFunctionTest;
+import org.apache.drill.test.BaseTestQuery;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+
+import static org.apache.drill.test.HadoopUtils.hadoopToJavaPath;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for CREATE DAFFODIL SCHEMA and DROP DAFFODIL SCHEMA SQL commands
+ */
+@Category({SlowTest.class, SqlFunctionTest.class})
+public class TestDaffodilSchemaHandlers extends BaseTestQuery {
+
+ private static final String DEFAULT_SCHEMA_JAR_NAME = "sample-daffodil-schema.jar";
+ private static URI fsUri;
+ private static File jarsDir;
+
+ @BeforeClass
+ public static void setupJars() throws IOException {
+ jarsDir = dirTestWatcher.makeSubDir(Paths.get("schema-jars"));
+ }
+
+ @Before
+ public void setupDrillbit() throws Exception {
+ updateTestCluster(1, config);
+ fsUri = getLocalFileSystem().getUri();
+ }
+
+ @After
+ public void cleanup() throws Exception {
+ closeClient();
+ dirTestWatcher.clear();
+ }
+
+ @Test
+ public void testCreateSyntax() throws Exception {
+ // Test that the SQL syntax is recognized
+ test("create daffodil schema using jar 'schema.jar'");
+ }
+
+ @Test
+ public void testDropSyntax() throws Exception {
+ // Test that the SQL syntax is recognized
+ test("drop daffodil schema using jar 'schema.jar'");
+ }
+
+ @Test
+ public void testCreateSchemaAbsentJarInStaging() throws Exception {
+ Path staging = hadoopToJavaPath(getDrillbitContext().getRemoteDaffodilSchemaRegistry().getStagingArea());
+
+ String summary = String.format("File %s does not exist on file system %s",
+ staging.resolve(DEFAULT_SCHEMA_JAR_NAME).toUri().getPath(), fsUri);
+
+ testBuilder()
+ .sqlQuery("create daffodil schema using jar '%s'", DEFAULT_SCHEMA_JAR_NAME)
+ .unOrdered()
+ .baselineColumns("ok", "summary")
+ .baselineValues(false, summary)
+ .go();
+ }
+
+ @Test
+ public void testDropSchemaNotRegistered() throws Exception {
+ String jarName = "non-existent-schema.jar";
+
+ testBuilder()
+ .sqlQuery("drop daffodil schema using jar '%s'", jarName)
+ .unOrdered()
+ .baselineColumns("ok", "summary")
+ .baselineValues(false, String.format("Jar %s is not registered in remote registry", jarName))
+ .go();
+ }
+
+ /**
+ * Test successful schema registration
+ */
+ @Test
+ public void testSuccessfulSchemaRegistration() throws Exception {
+ String jarName = "test-schema.jar";
+ File schemaJar = createTestSchemaJar(jarName);
+
+ Path staging = hadoopToJavaPath(getDrillbitContext().getRemoteDaffodilSchemaRegistry().getStagingArea());
+ copyJar(jarsDir.toPath(), staging, jarName);
+
+ testBuilder()
+ .sqlQuery("create daffodil schema using jar '%s'", jarName)
+ .unOrdered()
+ .baselineColumns("ok", "summary")
+ .baselineValues(true, String.format("Daffodil schema jar %s has been registered successfully.", jarName))
+ .go();
+ }
+
+ /**
+ * Test registering a duplicate schema JAR
+ */
+ @Test
+ public void testDuplicateSchemaRegistration() throws Exception {
+ String jarName = "duplicate-schema.jar";
+ File schemaJar = createTestSchemaJar(jarName);
+
+ Path staging = hadoopToJavaPath(getDrillbitContext().getRemoteDaffodilSchemaRegistry().getStagingArea());
+ copyJar(jarsDir.toPath(), staging, jarName);
+
+ // First registration should succeed
+ test("create daffodil schema using jar '%s'", jarName);
+
+ // Copy to staging again for second attempt
+ copyJar(jarsDir.toPath(), staging, jarName);
+
+ // Second registration should fail
+ testBuilder()
+ .sqlQuery("create daffodil schema using jar '%s'", jarName)
+ .unOrdered()
+ .baselineColumns("ok", "summary")
+ .baselineValues(false, String.format("Jar with %s name has been already registered", jarName))
+ .go();
+ }
+
+ /**
+ * Test successful schema unregistration
+ */
+ @Test
+ public void testSuccessfulSchemaUnregistration() throws Exception {
+ String jarName = "unregister-test-schema.jar";
+ File schemaJar = createTestSchemaJar(jarName);
+
+ Path staging = hadoopToJavaPath(getDrillbitContext().getRemoteDaffodilSchemaRegistry().getStagingArea());
+ copyJar(jarsDir.toPath(), staging, jarName);
+
+ // Register the schema
+ test("create daffodil schema using jar '%s'", jarName);
+
+ // Unregister the schema
+ testBuilder()
+ .sqlQuery("drop daffodil schema using jar '%s'", jarName)
+ .unOrdered()
+ .baselineColumns("ok", "summary")
+ .baselineValues(true, String.format("Daffodil schema jar %s has been unregistered successfully.", jarName))
+ .go();
+ }
+
+ /**
+ * Test concurrent access to same JAR during registration
+ */
+ @Test
+ public void testConcurrentRegistration() throws Exception {
+ String jarName = "concurrent-schema.jar";
+ File schemaJar = createTestSchemaJar(jarName);
+
+ Path staging = hadoopToJavaPath(getDrillbitContext().getRemoteDaffodilSchemaRegistry().getStagingArea());
+ copyJar(jarsDir.toPath(), staging, jarName);
+
+ // First attempt to register
+ test("create daffodil schema using jar '%s'", jarName);
+
+ // Attempting to register while it's already registered should indicate it's in use
+ copyJar(jarsDir.toPath(), staging, jarName);
+
+ testBuilder()
+ .sqlQuery("create daffodil schema using jar '%s'", jarName)
+ .unOrdered()
+ .baselineColumns("ok", "summary")
+ .baselineValues(false, String.format("Jar with %s name has been already registered", jarName))
+ .go();
+ }
+
+ /**
+ * Test registering and then dropping a schema in sequence
+ */
+ @Test
+ public void testRegisterAndDropSequence() throws Exception {
+ String jarName = "sequence-test-schema.jar";
+ File schemaJar = createTestSchemaJar(jarName);
+
+ Path staging = hadoopToJavaPath(getDrillbitContext().getRemoteDaffodilSchemaRegistry().getStagingArea());
+ Path registry = hadoopToJavaPath(getDrillbitContext().getRemoteDaffodilSchemaRegistry().getRegistryArea());
+
+ copyJar(jarsDir.toPath(), staging, jarName);
+
+ // Register
+ test("create daffodil schema using jar '%s'", jarName);
+
+ // Verify JAR is in registry
+ File registeredJar = registry.resolve(jarName).toFile();
+ assertTrue("JAR should exist in registry after registration", registeredJar.exists());
+
+ // Drop
+ test("drop daffodil schema using jar '%s'", jarName);
+
+ // Verify JAR is removed from registry
+ assertFalse("JAR should be removed from registry after dropping", registeredJar.exists());
+ }
+
+ /**
+ * Helper method to copy a jar file to a destination
+ */
+ private void copyJar(Path source, Path destination, String jarName) throws IOException {
+ FileUtils.copyFileToDirectory(source.resolve(jarName).toFile(), destination.toFile());
+ }
+
+ /**
+ * Helper method to create a simple JAR file for testing
+ * In a real implementation, this would create a JAR containing DFDL schema files
+ */
+ private File createTestSchemaJar(String jarName) throws IOException {
+ File jarFile = new File(jarsDir, jarName);
+ // Create an empty JAR file for basic testing
+ // In a real test, this would contain actual DFDL schema files (.xsd or .bin)
+ FileUtils.touch(jarFile);
+ return jarFile;
+ }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/schema/daffodil/TestRemoteDaffodilSchemaRegistry.java b/exec/java-exec/src/test/java/org/apache/drill/exec/schema/daffodil/TestRemoteDaffodilSchemaRegistry.java
new file mode 100644
index 00000000000..589d0833988
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/schema/daffodil/TestRemoteDaffodilSchemaRegistry.java
@@ -0,0 +1,101 @@
+/*
+ * 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.schema.daffodil;
+
+import org.apache.drill.categories.SlowTest;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.test.BaseDirTestWatcher;
+import org.apache.drill.test.ClusterTest;
+import org.apache.drill.test.ConfigBuilder;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.io.File;
+import java.nio.file.Paths;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+
+@Category({SlowTest.class})
+public class TestRemoteDaffodilSchemaRegistry extends ClusterTest {
+
+ @Rule
+ public final BaseDirTestWatcher dirTestWatcher = new BaseDirTestWatcher();
+
+ private RemoteDaffodilSchemaRegistry registry;
+ private DrillConfig config;
+ private File tempDir;
+
+ @Before
+ public void setup() throws Exception {
+ tempDir = dirTestWatcher.makeSubDir(Paths.get("daffodil-test"));
+
+ // Create a test configuration
+ config = new ConfigBuilder()
+ .put(ExecConstants.DFDL_DIRECTORY_ROOT, tempDir.getAbsolutePath())
+ .put(ExecConstants.DFDL_DIRECTORY_STAGING, tempDir.getAbsolutePath() + "/staging")
+ .put(ExecConstants.DFDL_DIRECTORY_REGISTRY, tempDir.getAbsolutePath() + "/registry")
+ .put(ExecConstants.DFDL_DIRECTORY_TMP, tempDir.getAbsolutePath() + "/tmp")
+ .build();
+ }
+
+ @After
+ public void cleanup() throws Exception {
+ if (registry != null) {
+ registry.close();
+ }
+ }
+
+ @Test
+ public void testInitialization() throws Exception {
+ // Note: This test would require a ClusterCoordinator and PersistentStoreProvider
+ // For now, we'll create a simple test that verifies the class can be instantiated
+ registry = new RemoteDaffodilSchemaRegistry();
+ assertNotNull("Registry should be instantiated", registry);
+ }
+
+ @Test
+ public void testRegistryHasRetryAttempts() throws Exception {
+ registry = new RemoteDaffodilSchemaRegistry();
+ // Before init, this might throw NPE or return 0
+ // After init with proper setup, it should return configured retry attempts
+ assertNotNull("Registry should be instantiated", registry);
+ }
+
+ @Test
+ public void testActionEnumValues() {
+ assertEquals("REGISTRATION action should exist",
+ RemoteDaffodilSchemaRegistry.Action.REGISTRATION,
+ RemoteDaffodilSchemaRegistry.Action.valueOf("REGISTRATION"));
+ assertEquals("UNREGISTRATION action should exist",
+ RemoteDaffodilSchemaRegistry.Action.UNREGISTRATION,
+ RemoteDaffodilSchemaRegistry.Action.valueOf("UNREGISTRATION"));
+ }
+
+ @Test
+ public void testHasRegistry() {
+ registry = new RemoteDaffodilSchemaRegistry();
+ // Before init, should return false
+ assertFalse("Registry should not exist before initialization", registry.hasRegistry());
+ }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/BaseTestQuery.java b/exec/java-exec/src/test/java/org/apache/drill/test/BaseTestQuery.java
index 79ef8119aa0..6320a2e388c 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/BaseTestQuery.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/BaseTestQuery.java
@@ -17,24 +17,8 @@
*/
package org.apache.drill.test;
-import static org.apache.drill.exec.util.StoragePluginTestUtils.ROOT_SCHEMA;
-import static org.apache.drill.exec.util.StoragePluginTestUtils.TMP_SCHEMA;
-import static org.hamcrest.core.StringContains.containsString;
-import static org.junit.Assert.assertEquals;
-import static org.hamcrest.MatcherAssert.assertThat;
-import static org.junit.Assert.fail;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.net.URL;
-import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Properties;
-import java.util.concurrent.atomic.AtomicInteger;
-
+import com.google.common.base.Preconditions;
+import com.google.common.io.Resources;
import org.apache.drill.common.config.DrillConfig;
import org.apache.drill.common.config.DrillProperties;
import org.apache.drill.common.exceptions.UserException;
@@ -65,8 +49,6 @@
import org.apache.drill.exec.util.StoragePluginTestUtils;
import org.apache.drill.exec.util.VectorUtil;
import org.apache.drill.exec.vector.ValueVector;
-import com.google.common.base.Preconditions;
-import com.google.common.io.Resources;
import org.apache.drill.test.DrillTestWrapper.TestServices;
import org.apache.hadoop.fs.FileSystem;
import org.junit.AfterClass;
@@ -74,6 +56,24 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.net.URL;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.apache.drill.exec.util.StoragePluginTestUtils.ROOT_SCHEMA;
+import static org.apache.drill.exec.util.StoragePluginTestUtils.TMP_SCHEMA;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.core.StringContains.containsString;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
/**
* deprecated Use {@link ClusterTest} instead.
*
@@ -168,6 +168,8 @@ protected static Properties cloneDefaultTestConfigProperties() {
props.setProperty(ExecConstants.SYS_STORE_PROVIDER_LOCAL_PATH, dirTestWatcher.getStoreDir().getAbsolutePath());
props.setProperty(ExecConstants.UDF_DIRECTORY_ROOT, dirTestWatcher.getHomeDir().getAbsolutePath());
props.setProperty(ExecConstants.UDF_DIRECTORY_FS, FileSystem.DEFAULT_FS);
+ props.setProperty(ExecConstants.DFDL_DIRECTORY_ROOT, dirTestWatcher.getHomeDir().getAbsolutePath());
+ props.setProperty(ExecConstants.DFDL_DIRECTORY_FS, FileSystem.DEFAULT_FS);
return props;
}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/ClusterFixture.java b/exec/java-exec/src/test/java/org/apache/drill/test/ClusterFixture.java
index de57903d692..4c1ae7e2be2 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/ClusterFixture.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/ClusterFixture.java
@@ -17,30 +17,13 @@
*/
package org.apache.drill.test;
-import static org.apache.drill.exec.util.StoragePluginTestUtils.DFS_TMP_SCHEMA;
-import static org.apache.drill.exec.util.StoragePluginTestUtils.ROOT_SCHEMA;
-import static org.apache.drill.exec.util.StoragePluginTestUtils.TMP_SCHEMA;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.URI;
-import java.net.URL;
-import java.nio.charset.StandardCharsets;
-import java.nio.file.Paths;
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.SQLException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Properties;
-
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.io.Resources;
import org.apache.drill.common.config.DrillProperties;
import org.apache.drill.common.logical.FormatPluginConfig;
import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.common.logical.security.PlainCredentialsProvider;
import org.apache.drill.exec.ExecConstants;
import org.apache.drill.exec.ZookeeperHelper;
import org.apache.drill.exec.client.DrillClient;
@@ -58,15 +41,32 @@
import org.apache.drill.exec.store.dfs.FileSystemConfig;
import org.apache.drill.exec.store.dfs.WorkspaceConfig;
import org.apache.drill.exec.store.mock.MockStorageEngineConfig;
-import org.apache.drill.common.logical.security.PlainCredentialsProvider;
import org.apache.drill.exec.store.sys.store.provider.ZookeeperPersistentStoreProvider;
import org.apache.drill.exec.util.StoragePluginTestUtils;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.io.Resources;
import org.apache.drill.test.DrillTestWrapper.TestServices;
import org.apache.hadoop.fs.FileSystem;
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URL;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+
+import static org.apache.drill.exec.util.StoragePluginTestUtils.DFS_TMP_SCHEMA;
+import static org.apache.drill.exec.util.StoragePluginTestUtils.ROOT_SCHEMA;
+import static org.apache.drill.exec.util.StoragePluginTestUtils.TMP_SCHEMA;
+
/**
* Test fixture to start a Drillbit with provide options, create a client, and
* execute queries. Can be used in JUnit tests, or in ad-hoc programs. Provides
@@ -615,8 +615,10 @@ public static ClusterFixtureBuilder builder(BaseDirTestWatcher dirTestWatcher) {
props.putAll(ClusterFixture.TEST_CONFIGURATIONS);
props.setProperty(ExecConstants.DRILL_TMP_DIR, dirTestWatcher.getTmpDir().getAbsolutePath());
props.setProperty(ExecConstants.UDF_DIRECTORY_ROOT, dirTestWatcher.getHomeDir().getAbsolutePath());
+ props.setProperty(ExecConstants.DFDL_DIRECTORY_ROOT, dirTestWatcher.getHomeDir().getAbsolutePath());
props.setProperty(ExecConstants.SYS_STORE_PROVIDER_LOCAL_PATH, dirTestWatcher.getStoreDir().getAbsolutePath());
props.setProperty(ExecConstants.UDF_DIRECTORY_FS, FileSystem.DEFAULT_FS);
+ props.setProperty(ExecConstants.DFDL_DIRECTORY_FS, FileSystem.DEFAULT_FS);
// ALTER SESSION profiles are seldom interesting
props.setProperty(ExecConstants.SKIP_ALTER_SESSION_QUERY_PROFILE, Boolean.TRUE.toString());
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/OperatorFixture.java b/exec/java-exec/src/test/java/org/apache/drill/test/OperatorFixture.java
index 2101da53aed..7547f22d0db 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/OperatorFixture.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/OperatorFixture.java
@@ -17,21 +17,10 @@
*/
package org.apache.drill.test;
-import org.apache.drill.exec.alias.AliasRegistryProvider;
-import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
-import org.apache.drill.exec.physical.resultSet.impl.ResultSetLoaderImpl;
-import org.apache.drill.exec.physical.resultSet.impl.ResultSetLoaderImpl.ResultSetOptions;
-import org.apache.drill.exec.server.DrillbitContext;
-import org.apache.drill.metastore.MetastoreRegistry;
import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import com.google.common.util.concurrent.ListenableFuture;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
import io.netty.buffer.DrillBuf;
import org.apache.calcite.schema.SchemaPlus;
import org.apache.drill.common.config.DrillConfig;
@@ -39,6 +28,7 @@
import org.apache.drill.common.scanner.persistence.ScanResult;
import org.apache.drill.common.types.TypeProtos;
import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.alias.AliasRegistryProvider;
import org.apache.drill.exec.compile.ClassBuilder;
import org.apache.drill.exec.compile.CodeCompiler;
import org.apache.drill.exec.exception.OutOfMemoryException;
@@ -56,6 +46,15 @@
import org.apache.drill.exec.ops.OperatorContext;
import org.apache.drill.exec.ops.OperatorStats;
import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.impl.ResultSetLoaderImpl;
+import org.apache.drill.exec.physical.resultSet.impl.ResultSetLoaderImpl.ResultSetOptions;
+import org.apache.drill.exec.physical.rowSet.DirectRowSet;
+import org.apache.drill.exec.physical.rowSet.HyperRowSetImpl;
+import org.apache.drill.exec.physical.rowSet.IndirectRowSet;
+import org.apache.drill.exec.physical.rowSet.RowSet;
+import org.apache.drill.exec.physical.rowSet.RowSet.ExtendableRowSet;
+import org.apache.drill.exec.physical.rowSet.RowSetBuilder;
import org.apache.drill.exec.planner.PhysicalPlanReaderTestFactory;
import org.apache.drill.exec.proto.ExecProtos;
import org.apache.drill.exec.record.BatchSchema;
@@ -65,6 +64,7 @@
import org.apache.drill.exec.record.metadata.MetadataUtils;
import org.apache.drill.exec.record.metadata.TupleMetadata;
import org.apache.drill.exec.record.selection.SelectionVector2;
+import org.apache.drill.exec.server.DrillbitContext;
import org.apache.drill.exec.server.options.OptionManager;
import org.apache.drill.exec.server.options.SystemOptionManager;
import org.apache.drill.exec.store.PartitionExplorer;
@@ -72,15 +72,16 @@
import org.apache.drill.exec.store.sys.store.provider.LocalPersistentStoreProvider;
import org.apache.drill.exec.testing.ExecutionControls;
import org.apache.drill.exec.work.filter.RuntimeFilterWritable;
+import org.apache.drill.metastore.MetastoreRegistry;
import org.apache.drill.test.ClusterFixtureBuilder.RuntimeOption;
-import org.apache.drill.exec.physical.rowSet.DirectRowSet;
-import org.apache.drill.exec.physical.rowSet.HyperRowSetImpl;
-import org.apache.drill.exec.physical.rowSet.IndirectRowSet;
-import org.apache.drill.exec.physical.rowSet.RowSet;
-import org.apache.drill.exec.physical.rowSet.RowSet.ExtendableRowSet;
-import org.apache.drill.exec.physical.rowSet.RowSetBuilder;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.security.UserGroupInformation;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
/**
@@ -137,6 +138,8 @@ public Builder(BaseDirTestWatcher dirTestWatcher)
configBuilder.put(ExecConstants.HASHJOIN_SPILL_DIRS, Arrays.asList(dirTestWatcher.getSpillDir().getAbsolutePath()));
configBuilder.put(ExecConstants.UDF_DIRECTORY_ROOT, dirTestWatcher.getHomeDir().getAbsolutePath());
configBuilder.put(ExecConstants.UDF_DIRECTORY_FS, FileSystem.DEFAULT_FS);
+ configBuilder.put(ExecConstants.DFDL_DIRECTORY_ROOT, dirTestWatcher.getHomeDir().getAbsolutePath());
+ configBuilder.put(ExecConstants.DFDL_DIRECTORY_FS, FileSystem.DEFAULT_FS);
}
}
diff --git a/exec/java-exec/src/test/resources/drill-spool-test-module.conf b/exec/java-exec/src/test/resources/drill-spool-test-module.conf
index a3c280d80e8..d18dd05f3cd 100644
--- a/exec/java-exec/src/test/resources/drill-spool-test-module.conf
+++ b/exec/java-exec/src/test/resources/drill-spool-test-module.conf
@@ -95,6 +95,16 @@ drill: {
impl: "org.apache.drill.exec.work.batch.SpoolingRawBatchBuffer",
delete: false,
size: 0
+ },
+ daffodil: {
+ retry-attempts: 5,
+ directory: {
+ base: "/tmp/drill/daffodil",
+ local: "/tmp/drill/daffodil/local",
+ staging: "/tmp/drill/daffodil/staging",
+ registry: "/tmp/drill/daffodil/registry",
+ tmp: "/tmp/drill/daffodil/tmp"
+ }
}
}
}
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/MapBuilder.java b/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/MapBuilder.java
index f333cb9cd2b..37809e64826 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/MapBuilder.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/MapBuilder.java
@@ -22,6 +22,8 @@
import org.apache.drill.common.types.TypeProtos.MinorType;
import org.apache.drill.exec.record.MaterializedField;
+import java.util.Objects;
+
/**
* Internal structure for building a map. A map is just a schema,
* but one that is part of a parent column.
@@ -33,7 +35,7 @@
* All resumeXXX methods do not produce any action and return null.
* To access built column {@link #buildColumn()} should be used.
*/
-public class MapBuilder implements SchemaContainer {
+public class MapBuilder implements MapBuilderLike, SchemaContainer {
private final SchemaContainer parent;
private final TupleBuilder tupleBuilder = new TupleBuilder();
private final String memberName;
@@ -68,6 +70,7 @@ public MapBuilder add(String name, MinorType type, DataMode mode) {
return this;
}
+ @Override
public MapBuilder add(String name, MinorType type) {
tupleBuilder.add(name, type);
return this;
@@ -82,6 +85,7 @@ public MapBuilder add(String name, MinorType type, int precision, int scale) {
return addDecimal(name, type, DataMode.REQUIRED, precision, scale);
}
+ @Override
public MapBuilder addNullable(String name, MinorType type) {
tupleBuilder.addNullable(name, type);
return this;
@@ -96,6 +100,7 @@ public MapBuilder addNullable(String name, MinorType type, int precision, int sc
return addDecimal(name, type, DataMode.OPTIONAL, precision, scale);
}
+ @Override
public MapBuilder addArray(String name, MinorType type) {
tupleBuilder.addArray(name, type);
return this;
@@ -129,10 +134,12 @@ public MapBuilder addDecimal(String name, MinorType type,
* @param name the name of the map column
* @return a builder for the map
*/
+ @Override
public MapBuilder addMap(String name) {
return tupleBuilder.addMap(this, name);
}
+ @Override
public MapBuilder addMapArray(String name) {
return tupleBuilder.addMapArray(this, name);
}
@@ -185,6 +192,27 @@ public MapBuilder resumeMap() {
return (MapBuilder) parent;
}
+ /**
+ * Depending on whether the parent is a schema builder or map builder
+ * we resume appropriately.
+ */
+ @Override
+ public void resume() {
+ if (Objects.isNull(parent)) {
+ throw new IllegalStateException("Call to resume() on MapBuilder with no parent.");
+ }
+ if (parent instanceof MapBuilder) {
+ resumeMap();
+ } else {
+ assert(parent instanceof SchemaBuilder);
+ //
+ // This would be extended for other kinds of possible containers of a Map.
+ // First version only needed SchemaBuilder parents
+ //
+ resumeSchema();
+ }
+ }
+
public RepeatedListBuilder resumeList() {
build();
return (RepeatedListBuilder) parent;
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/MapBuilderLike.java b/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/MapBuilderLike.java
new file mode 100644
index 00000000000..086c6a5b5e7
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/MapBuilderLike.java
@@ -0,0 +1,41 @@
+/*
+ * 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.record.metadata;
+
+import org.apache.drill.common.types.TypeProtos;
+
+/**
+ * A common interface shared by SchemaBuilder and MapBuilder allowing one to do most
+ * operations for constructing metadata for hierarchical data
+ * without having to keep track of whether you are dealing with the top-level schema/row
+ * level or a map within it.
+ */
+public interface MapBuilderLike {
+
+ MapBuilderLike addArray(String colName, TypeProtos.MinorType drillType);
+
+ MapBuilderLike addNullable(String colName, TypeProtos.MinorType drillType);
+
+ MapBuilderLike add(String colName, TypeProtos.MinorType drillType);
+
+ MapBuilderLike addMapArray(String colName);
+
+ MapBuilderLike addMap(String colName);
+
+ void resume();
+}
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/SchemaBuilder.java b/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/SchemaBuilder.java
index c1b97609357..f2aee1d2190 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/SchemaBuilder.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/SchemaBuilder.java
@@ -61,7 +61,7 @@
*
*/
-public class SchemaBuilder implements SchemaContainer {
+public class SchemaBuilder implements MapBuilderLike, SchemaContainer {
/**
* Actual tuple schema builder. The odd layered structure is needed
@@ -114,6 +114,7 @@ public SchemaBuilder add(String name, MinorType type, DataMode mode) {
return this;
}
+ @Override
public SchemaBuilder add(String name, MinorType type) {
tupleBuilder.add(name, type);
return this;
@@ -128,6 +129,7 @@ public SchemaBuilder add(String name, MinorType type, int precision, int scale)
return addDecimal(name, type, DataMode.REQUIRED, precision, scale);
}
+ @Override
public SchemaBuilder addNullable(String name, MinorType type) {
tupleBuilder.addNullable(name, type);
return this;
@@ -142,6 +144,7 @@ public SchemaBuilder addNullable(String name, MinorType type, int precision, int
return addDecimal(name, type, DataMode.OPTIONAL, precision, scale);
}
+ @Override
public SchemaBuilder addArray(String name, MinorType type) {
tupleBuilder.addArray(name, type);
return this;
@@ -157,7 +160,7 @@ public SchemaBuilder addDecimal(String name, MinorType type, DataMode mode, int
}
/**
- * Add a multi-dimensional array, implemented as a repeated vector
+ * Add a multidimensional array, implemented as a repeated vector
* along with 0 or more repeated list vectors.
*
* @param name column name
@@ -191,10 +194,12 @@ public SchemaBuilder addAll(TupleMetadata from) {
* @param name the name of the map column
* @return a builder for the map
*/
+ @Override
public MapBuilder addMap(String name) {
return tupleBuilder.addMap(this, name);
}
+ @Override
public MapBuilder addMapArray(String name) {
return tupleBuilder.addMapArray(this, name);
}
@@ -237,4 +242,9 @@ public TupleMetadata buildSchema() {
public TupleMetadata build() {
return tupleBuilder.schema();
}
+
+ @Override
+ public void resume() {
+ // do nothing. There is nothing else to resume.
+ }
}
diff --git a/pom.xml b/pom.xml
index f229eb67d0f..f3a79ca03fe 100644
--- a/pom.xml
+++ b/pom.xml
@@ -71,6 +71,7 @@
1.10.0
1.7
5.5.0
+ 3.11.0
10.17.1.0
3072
apache/drill