You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by GitBox <gi...@apache.org> on 2021/07/29 18:52:31 UTC

[GitHub] [drill] MFoss19 opened a new pull request #2282: DRILL-7978: Fixed Width Format Plugin

MFoss19 opened a new pull request #2282:
URL: https://github.com/apache/drill/pull/2282


   # [DRILL-7978](https://issues.apache.org/jira/browse/DRILL-7978): Fixed Width Format Plugin
   
   ## Description
   Developing format plugin to parse fixed width files.
   
   Fixed Width Text File Definition: https://www.oracle.com/webfolder/technetwork/data-quality/edqhelp/Content/introduction/getting_started/configuring_fixed_width_text_file_formats.htm
   
   ## Documentation
   Users can now create a format configuration to parse fixed width files.
   
   ## Testing
   Unit tests added. More to come.
   


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

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

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



[GitHub] [drill] lgtm-com[bot] commented on pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #2282:
URL: https://github.com/apache/drill/pull/2282#issuecomment-954232588


   This pull request **introduces 2 alerts** when merging 05ae3f1dcadfa067a569fd0634f487e5e282d83e into 58ced604ae460a72b73c85dd97bb3a7df756dcbf - [view on LGTM.com](https://lgtm.com/projects/g/apache/drill/rev/pr-14a39795f41d4ea8aeb89a462e761ba10f4b37f3)
   
   **new alerts:**
   
   * 2 for Unused format argument


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

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

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



[GitHub] [drill] dzamo commented on pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
dzamo commented on pull request #2282:
URL: https://github.com/apache/drill/pull/2282#issuecomment-992496691


   Hi @paul-rogers.  We're in the throes of trying to convert this plugin to use EVF v2 / scan.v3.  This will be the first instance of this kind in the Drill code base, apart from a very simple mock plugin which supports unit tests (CompliantTextBatchReader remains based on EVF v1, from what I can see).
   
   Something that's confusing me is that the EasyFormatPlugin base class is coded against the ManagedReader interface from EVF v1.  So I cannot see that we can both derive from EasyFormatPlugin, and also implement ManagedReader from EVF v2.  Am I missing something here?
   
   Thanks, James


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

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

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



[GitHub] [drill] MFoss19 commented on a change in pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
MFoss19 commented on a change in pull request #2282:
URL: https://github.com/apache/drill/pull/2282#discussion_r686053013



##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthFieldConfig.java
##########
@@ -0,0 +1,71 @@
+package org.apache.drill.exec.store.fixedwidth;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.drill.common.types.TypeProtos;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName("fixedwidthReaderFieldDescription")
+@JsonInclude(JsonInclude.Include.NON_DEFAULT)

Review comment:
       Done




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

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

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



[GitHub] [drill] estherbuchwalter commented on a change in pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
estherbuchwalter commented on a change in pull request #2282:
URL: https://github.com/apache/drill/pull/2282#discussion_r757612176



##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthFormatConfig.java
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.fixedwidth;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.common.PlanStringBuilder;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.logical.FormatPluginConfig;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.regex.Pattern;
+
+@JsonTypeName(FixedwidthFormatPlugin.DEFAULT_NAME)
+@JsonInclude(JsonInclude.Include.NON_DEFAULT)
+public class FixedwidthFormatConfig implements FormatPluginConfig {
+  private static final Logger logger = LoggerFactory.getLogger(FixedwidthFormatConfig.class);
+  private final List<String> extensions;
+  private final List<FixedwidthFieldConfig> fields;
+  private final List<TypeProtos.MinorType> validDataTypes = Arrays.asList(TypeProtos.MinorType.INT, TypeProtos.MinorType.VARCHAR,
+    TypeProtos.MinorType.DATE, TypeProtos.MinorType.TIME, TypeProtos.MinorType.TIMESTAMP, TypeProtos.MinorType.FLOAT4,
+    TypeProtos.MinorType.FLOAT8, TypeProtos.MinorType.BIGINT, TypeProtos.MinorType.VARDECIMAL);
+
+  @JsonCreator
+  public FixedwidthFormatConfig(@JsonProperty("extensions") List<String> extensions,
+                                @JsonProperty("fields") List<FixedwidthFieldConfig> fields) {
+    this.extensions = extensions == null ? Collections.singletonList("fwf") : ImmutableList.copyOf(extensions);
+    Collections.sort(fields);
+    this.fields = fields;
+
+    validateFieldInput();
+  }
+
+  @JsonInclude(JsonInclude.Include.NON_DEFAULT)
+  public List<String> getExtensions() {
+    return extensions;
+  }
+
+  public List<FixedwidthFieldConfig> getFields() {
+    return fields;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(extensions, fields);
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null || getClass() != obj.getClass()) {
+      return false;
+    }
+    FixedwidthFormatConfig other = (FixedwidthFormatConfig) obj;
+    return Objects.equals(extensions, other.extensions)
+            && Objects.equals(fields, other.fields);
+  }
+
+  @Override
+  public String toString() {
+    return new PlanStringBuilder(this)
+            .field("extensions", extensions)
+            .field("fields", fields)
+            .toString();
+  }
+
+
+  @JsonIgnore
+  public boolean hasFields() {
+    return fields != null && ! fields.isEmpty();
+  }
+
+  @JsonIgnore
+  public List<String> getFieldNames() {
+    List<String> result = new ArrayList<>();
+    if (! hasFields()) {
+      return result;
+    }
+
+    for (FixedwidthFieldConfig field : fields) {
+      result.add(field.getName());
+    }
+    return result;
+  }
+
+  @JsonIgnore
+  public List<Integer> getFieldIndices() {
+    List<Integer> result = new ArrayList<>();
+    if (! hasFields()) {
+      return result;
+    }
+
+    for (FixedwidthFieldConfig field : fields) {
+      result.add(field.getIndex());
+    }
+    return result;
+  }
+
+  @JsonIgnore
+  public List<Integer> getFieldWidths() {
+    List<Integer> result = new ArrayList<>();
+    if (! hasFields()) {
+      return result;
+    }
+
+    for (FixedwidthFieldConfig field : fields) {
+      result.add(field.getWidth());
+    }
+    return result;
+  }
+
+  @JsonIgnore
+  public List<TypeProtos.MinorType> getFieldTypes() {
+    List<TypeProtos.MinorType> result = new ArrayList<>();
+    if (! hasFields()) {
+      return result;
+    }
+
+    for (FixedwidthFieldConfig field : fields) {
+      result.add(field.getType());
+    }
+    return result;
+  }
+
+  @JsonIgnore
+  public void setFieldTypes(int i) {
+    for (FixedwidthFieldConfig field : fields) {
+      if (field.getIndex() == i) {
+        field.setType();
+      }
+    }
+  }
+
+  @JsonIgnore
+  public void validateFieldInput(){
+    Set<String> uniqueNames = new HashSet<>();
+    List<Integer> fieldIndices = this.getFieldIndices();
+    List<Integer> fieldWidths = this.getFieldWidths();
+    List<String> fieldNames = this.getFieldNames();
+    List<TypeProtos.MinorType> fieldTypes = this.getFieldTypes();
+    int prevIndexAndWidth = -1;
+
+    /* Validate Field Name - Ensure field is not empty, does not exceed maximum length,
+    is valid SQL syntax, and no two fields have the same name
+     */
+    for (String name : this.getFieldNames()){
+      if (name.length() == 0){
+        throw UserException
+          .validationError()
+          .message("Blank field name detected.")
+          .addContext("Plugin", FixedwidthFormatPlugin.DEFAULT_NAME)
+          .build(logger);
+      }
+      if (name.length() > 1024) {
+        throw UserException
+          .validationError()
+          .message("Exceeds maximum length of 1024 characters: " + name.substring(0, 1024))
+          .addContext("Plugin", FixedwidthFormatPlugin.DEFAULT_NAME)
+          .build(logger);
+      }
+      if (!Pattern.matches("[a-zA-Z]\\w*", name)) {
+        throw UserException
+          .validationError()
+          .message("Invalid input: " + name)

Review comment:
       The regex here says that the first character must be a letter, and the rest (\w*) must be alphabetical, numerical, or the underscore.




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

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

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



[GitHub] [drill] paul-rogers commented on pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
paul-rogers commented on pull request #2282:
URL: https://github.com/apache/drill/pull/2282#issuecomment-1003835781


   PR [2419](https://github.com/apache/drill/pull/2419) has the EVF V2 adapter for the Easy Format Plugin. I suggest that you use that code, and follow the example there, to add EVF V2 support here.


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

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

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



[GitHub] [drill] lgtm-com[bot] commented on pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #2282:
URL: https://github.com/apache/drill/pull/2282#issuecomment-972148125


   This pull request **introduces 2 alerts** when merging f9e96fec406d2ce48810fb1f25291c7d54dcdcfb into 42e7b77b302805156c48952e9e6a7b5b5383dfdc - [view on LGTM.com](https://lgtm.com/projects/g/apache/drill/rev/pr-daf68224b7b9e6213431bcad29255525dd11b8ac)
   
   **new alerts:**
   
   * 2 for Unused format argument


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

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

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



[GitHub] [drill] cgivre commented on pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
cgivre commented on pull request #2282:
URL: https://github.com/apache/drill/pull/2282#issuecomment-963130138


   > @MFoss19 @estherbuchwalter following some [recent chat](https://github.com/apache/drill/pull/2359#issuecomment-962673076) with @paul-rogers and my last comment here, how about a reduced format config such as the following? The goal is to get to something terse and consistent with what we do for other text formats.
   > 
   > ```json
   > "fixedwidth": {
   >   "type": "fixedwidth",
   >   "extensions": [
   >     "fwf"
   >   ],
   >   "extractHeader": true,
   >   "trimStrings": true,
   >   "columnOffsets": [1, 11, 21, 31],
   >   "columnWidths": [10, 10, 10, 10]
   > }
   > ```
   > 
   > Column names and types can already come from a provided schema or aliasing after calls to `CAST()`. Incidentally, the settings above can be overriden per query using a provided schema too.
   > 
   > There's also a part of that wonders whether we could have justified adding our fixed width functionality to the existing delimited text format reader.
   
   @dzamo In this case, I'd respectfully disagree here.  In effect, the configuration is providing a schema to the user, similar to the way the logRegex reader works.  In this case, the user will get the best data possible if we can include datatypes and field names in the schema, so that they can just do a `SELECT *` and not have to worry about casting etc. 
   
   Let's consider a real world use case: some fixed width log generated by a database.  Since the fields may be mashed together, there isn't a delimiter that you can use to divide the fields.   You *could* use however the logRegex reader to do this.  That point aside for the moment, the way I imagined someone using this was that different configs could be set up and linked to workspaces such that if a file was in the `mysql_logs` folder, it would use the mysql log config, and if it was in the `postgres` it would use another.  
   
   My opinion here is that the goal should be to get the cleanest data to the user as possible without the user having to rely on CASTs and other complicating factors. 


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

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

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



[GitHub] [drill] MFoss19 commented on a change in pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
MFoss19 commented on a change in pull request #2282:
URL: https://github.com/apache/drill/pull/2282#discussion_r759503228



##########
File path: contrib/format-fixedwidth/src/test/resources/fwf/test_blankrow.fwf
##########
@@ -0,0 +1,26 @@
+12.34 test 567 02-10-2021 10:30:27 02-10-2021T10:30:27.00Z
+56.78 TEST 890 07-27-2021 12:40:15 07-27-2021T12:40:15.00Z
+11.11 abcd 111 11-11-1111 11:11:11 11-11-1111T11:11:11.11Z
+22.22 efgh 222 01-22-2222 22:22:22 01-22-2222T22:22:22.22Z
+33.33 ijkl 333 02-01-3333 01:33:33 02-01-3333T01:33:33.33Z
+44.44 mnop 444 03-02-4444 02:44:44 03-02-4444T02:44:44.44Z
+55.55 qrst 555 04-03-5555 03:55:55 04-03-5555T03:55:55.55Z
+66.66 uvwx 666 05-04-6666 04:01:01 05-04-6666T04:01:01.01Z
+77.77 yzzz 777 06-05-7777 05:11:11 06-05-7777T05:11:11.11Z
+88.88 aabb 888 07-06-8888 06:22:22 07-07-8888T06:22:22.22Z
+88.88 aabb 888 07-06-8888 06:22:22 07-07-8888T06:22:22.22Z
+

Review comment:
       It is intentional to test what would happen if there was a blank line in a user input file. Good idea to add the explanation.txt file and additional invalid files.




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

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

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



[GitHub] [drill] dzamo commented on a change in pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
dzamo commented on a change in pull request #2282:
URL: https://github.com/apache/drill/pull/2282#discussion_r742913804



##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,200 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.math.BigDecimal;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(FixedwidthBatchReader.class);
+  private FileSplit split;
+  private final int maxRecords;
+  private final FixedwidthFormatConfig config;
+  private CustomErrorContext errorContext;
+  private InputStream fsStream;
+  private ResultSetLoader loader;
+  private RowSetLoader writer;
+  private BufferedReader reader;
+  private int lineNum;
+
+  public FixedwidthBatchReader(FixedwidthFormatConfig config, int maxRecords) {
+    this.config = config;
+    this.maxRecords = maxRecords;
+  }
+
+
+  @Override
+  public boolean open(FileSchemaNegotiator negotiator) {
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    lineNum = 0;
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      negotiator.tableSchema(buildSchema(), true);
+      loader = negotiator.build();
+    } catch (Exception e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to open input file: {}", split.getPath().toString())
+        .addContext(errorContext)
+        .addContext(e.getMessage())
+        .build(logger);
+    }
+    reader = new BufferedReader(new InputStreamReader(fsStream, Charsets.UTF_8));
+    return true;
+  }
+
+  @Override
+  public boolean next() { // Use loader to read data from file to turn into Drill rows
+    String line;
+    RowSetLoader writer = loader.writer();
+
+    try {
+      line = reader.readLine();
+      while (!writer.isFull() && line != null) {
+        writer.start();
+        parseLine(line, writer);
+        writer.save();
+        line = reader.readLine();
+        lineNum++;
+      }
+    } catch (IOException e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to read input file: {}", split.getPath().toString())
+        .addContext(errorContext)
+        .addContext(e.getMessage())
+        .addContext("Line Number", lineNum)
+        .build(logger);
+    }
+    return writer.limitReached(maxRecords);  // returns false when maxRecords limit has been reached
+  }
+
+  @Override
+  public void close() {
+    if (fsStream != null){
+      AutoCloseables.closeSilently(fsStream);
+      fsStream = null;
+    }
+  }
+
+  private TupleMetadata buildSchema() {
+    SchemaBuilder builder = new SchemaBuilder();
+    for (FixedwidthFieldConfig field : config.getFields()) {
+      if (field.getType() == TypeProtos.MinorType.VARDECIMAL){
+        builder.addNullable(field.getName(), TypeProtos.MinorType.VARDECIMAL,38,4);
+        //revisit this
+      } else {
+        builder.addNullable(field.getName(), field.getType());
+      }
+    }
+    return builder.buildSchema();
+  }
+
+
+  private boolean parseLine(String line, RowSetLoader writer) throws IOException {
+    int i = 0;
+    TypeProtos.MinorType dataType;
+    String dateTimeFormat;
+    String value;
+    for (FixedwidthFieldConfig field : config.getFields()) {

Review comment:
       @cgivre @MFoss19 @estherbuchwalter here we are reading column data types from the format config, where we also specify their start and stop offsets.  But this format plugin can also accept data types from a provided schema.  So my question is: can we drop the data type information from the format config so that we don't introduce multiple ad-hoc ways of specifying this info?  This is genuinely a question because I don't know this subject well...




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

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

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



[GitHub] [drill] paul-rogers commented on a change in pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
paul-rogers commented on a change in pull request #2282:
URL: https://github.com/apache/drill/pull/2282#discussion_r692565008



##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthFieldConfig.java
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.fixedwidth;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.common.types.TypeProtos;
+
+@JsonTypeName("fixedwidthReaderFieldDescription")
+@JsonInclude(JsonInclude.Include.NON_DEFAULT)
+public class FixedwidthFieldConfig {
+
+  private final TypeProtos.MinorType dataType;
+  private final String fieldName;
+  private final String dateTimeFormat;
+  private final int startIndex;
+  private final int fieldWidth;
+
+  public FixedwidthFieldConfig(@JsonProperty("dataType") TypeProtos.MinorType dataType,

Review comment:
       Does it work to use the `MinorType` here? Is that type set up for Jackson serialization? I don't know the answer, just noting we should double-check to ensure it works OK.

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthFieldConfig.java
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.fixedwidth;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.common.types.TypeProtos;
+
+@JsonTypeName("fixedwidthReaderFieldDescription")
+@JsonInclude(JsonInclude.Include.NON_DEFAULT)
+public class FixedwidthFieldConfig {
+
+  private final TypeProtos.MinorType dataType;
+  private final String fieldName;
+  private final String dateTimeFormat;
+  private final int startIndex;
+  private final int fieldWidth;
+
+  public FixedwidthFieldConfig(@JsonProperty("dataType") TypeProtos.MinorType dataType,
+                               @JsonProperty("fieldName") String fieldName,
+                               @JsonProperty("dateTimeFormat") String dateTimeFormat,
+                               @JsonProperty("startIndex") int startIndex,
+                               @JsonProperty("fieldWidth") int fieldWidth) {
+    this.dataType = dataType;
+    this.fieldName = fieldName;
+    this.dateTimeFormat = dateTimeFormat;
+    this.startIndex = startIndex;
+    this.fieldWidth = fieldWidth;

Review comment:
       Since configs are created by hand, having good defaults is helpful. Perhaps:
   
   * `name`: required; throw an exception if blank, or if the stripped name is not a valid SQL symbol.
   * `type`: default to `VARCHAR`
   * `dateTimeFormat`: `null` is allowed, so no default.
   * `index`: required, must be >= 0.
   * `width`: either required, or can be optional. If provided must be > 0. (See below.)
   
   For this plugin, we also have to check the set of fields.
   
   * No two fields can have the same name.
   * Should fields be allowed to overlap?
   
   We could be clever. Scan all fields and sort into ascending order. If a field omits the width, just compute it from the index of this and the next field.
   

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator> {

Review comment:
       This uses "EVF V1". Your plugin provides schema information, and is thus a perfect fit for "EVF V2" which can use your schema information to set up the row set loader schema automatically for you.

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthFieldConfig.java
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.fixedwidth;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.common.types.TypeProtos;
+
+@JsonTypeName("fixedwidthReaderFieldDescription")
+@JsonInclude(JsonInclude.Include.NON_DEFAULT)
+public class FixedwidthFieldConfig {
+
+  private final TypeProtos.MinorType dataType;
+  private final String fieldName;
+  private final String dateTimeFormat;
+  private final int startIndex;
+  private final int fieldWidth;
+
+  public FixedwidthFieldConfig(@JsonProperty("dataType") TypeProtos.MinorType dataType,
+                               @JsonProperty("fieldName") String fieldName,
+                               @JsonProperty("dateTimeFormat") String dateTimeFormat,
+                               @JsonProperty("startIndex") int startIndex,
+                               @JsonProperty("fieldWidth") int fieldWidth) {

Review comment:
       Drill has no config builder UI, users have to create JSON configs by hand. We've found it is helpful to keep field names short (Go-style.) So, perhaps:
   
   * `dataType` &rarr; `type`
   * `fieldName` &rarr; `name`
   * `dateTimeFormat` long, but OK
   * `startIndex` &rarr; `index`
   * `fieldWidth` &rarr; `width`
   
   Note that, since this is a `FieldConfig` we don't need the `field` prefix.
   
   Also, it can help readers if the "primary key" fields are first, so perhaps change the order to
   
   * `name`
   * `index`
   * `width`
   * `type`
   * `dateTimeFormat`

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthFormatPlugin.java
##########
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.drill.exec.store.fixedwidth;
+
+import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileReaderFactory;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileScanBuilder;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.options.OptionManager;
+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 FixedwidthFormatPlugin extends EasyFormatPlugin<FixedwidthFormatConfig> {
+
+  protected static final String DEFAULT_NAME = "fixedwidth";
+
+  private static class FixedwidthReaderFactory extends FileReaderFactory {
+
+    private final FixedwidthFormatConfig config;
+    private final int maxRecords;
+
+    public FixedwidthReaderFactory(FixedwidthFormatConfig config, int maxRecords) {
+      this.config = config;
+      this.maxRecords = maxRecords;
+    }
+
+    @Override
+    public ManagedReader<? extends FileSchemaNegotiator> newReader() {
+      return new FixedwidthBatchReader(config, maxRecords);
+    }
+  }
+
+  public FixedwidthFormatPlugin(String name,
+                                DrillbitContext context,
+                                Configuration fsConf,
+                                StoragePluginConfig storageConfig,
+                                FixedwidthFormatConfig formatConfig) {
+    super(name, easyConfig(fsConf, formatConfig), context, storageConfig, formatConfig);
+  }
+
+  private static EasyFormatConfig easyConfig(Configuration fsConf, FixedwidthFormatConfig pluginConfig) {
+    return EasyFormatConfig.builder()
+      .readable(true)
+      .writable(false)
+      .blockSplittable(false)

Review comment:
       I'm pretty sure fix-width files are splittable. If every records resides on a single line, they the file is spittable if we add code that, if the start offset !=0, scan to the next newline. And, on read, read rows until the file position is greater than the block end. See the text file (CSV) plugin for details (though, don't follow its implementation as that implementation is rather unique to that one use case.)

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthFormatConfig.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.store.fixedwidth;
+
+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 org.apache.drill.common.PlanStringBuilder;
+import org.apache.drill.common.logical.FormatPluginConfig;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+@JsonTypeName(FixedwidthFormatPlugin.DEFAULT_NAME)
+@JsonInclude(JsonInclude.Include.NON_DEFAULT)
+public class FixedwidthFormatConfig implements FormatPluginConfig {
+  private final List<String> extensions;
+  private final List<FixedwidthFieldConfig> fields;
+
+  @JsonCreator
+  public FixedwidthFormatConfig(@JsonProperty("extensions") List<String> extensions,
+                                @JsonProperty("fields") List<FixedwidthFieldConfig> fields) {
+    this.extensions = extensions == null ? Collections.singletonList("fwf") : ImmutableList.copyOf(extensions);
+    this.fields = fields;
+  }
+
+  @JsonInclude(JsonInclude.Include.NON_DEFAULT)
+  public List<String> getExtensions() {
+    return extensions;
+  }
+
+  public List<FixedwidthFieldConfig> getFields() {
+    return fields;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(extensions, fields);
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null || getClass() != obj.getClass()) {
+      return false;
+    }
+    FixedwidthFormatConfig other = (FixedwidthFormatConfig) obj;
+    return Objects.equals(extensions, other.extensions)
+            && Objects.equals(fields, other.fields);
+  }
+
+  @Override
+  public String toString() {
+    return new PlanStringBuilder(this)
+            .field("extensions", extensions)
+            .field("fields", fields)
+            .toString();
+  }
+}

Review comment:
       Nit: GitHub is complaining about a lack of final newlines.

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(FixedwidthBatchReader.class);
+  private FileSplit split;
+  private final int maxRecords;
+  private final FixedwidthFormatConfig config;
+  private CustomErrorContext errorContext;
+  private InputStream fsStream;
+  private ResultSetLoader loader;
+  private RowSetLoader writer;
+  private BufferedReader reader;
+  private int lineNum;
+
+  public FixedwidthBatchReader(FixedwidthFormatConfig config, int maxRecords) {
+    this.config = config;
+    this.maxRecords = maxRecords;
+  }
+
+  @Override
+  public boolean open(FileSchemaNegotiator negotiator) {
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    lineNum = 0;
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      negotiator.tableSchema(buildSchema(), true);
+      loader = negotiator.build();
+    } catch (Exception e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to open input file: {}", split.getPath().toString())
+        .addContext(errorContext)
+        .addContext(e.getMessage())
+        .build(logger);
+    }
+    reader = new BufferedReader(new InputStreamReader(fsStream, Charsets.UTF_8));
+    return true;
+  }
+
+  @Override
+  public boolean next() { // Use loader to read data from file to turn into Drill rows
+    String line;
+    RowSetLoader writer = loader.writer();
+
+    try {
+      line = reader.readLine();
+      while (!writer.isFull() && line != null) {
+        writer.start();
+        parseLine(line, writer);
+        writer.save();
+        line = reader.readLine();
+        lineNum++;
+      }
+    } catch (IOException e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to read input file: {}", split.getPath().toString())
+        .addContext(errorContext)
+        .addContext(e.getMessage())
+        .addContext("Line Number", lineNum)
+        .build(logger);
+    }
+    return writer.limitReached(maxRecords);  // returns false when maxRecords limit has been reached
+  }
+
+  @Override
+  public void close() {
+    try {
+      fsStream.close();
+      loader.close();
+    } catch (Exception e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to close input file: {}", split.getPath().toString())
+        .addContext(errorContext)
+        .addContext(e.getMessage())
+        .build(logger);
+    }
+  }
+
+  private TupleMetadata buildSchema() {
+    SchemaBuilder builder = new SchemaBuilder();
+    for (FixedwidthFieldConfig field : config.getFields()) {
+      builder.addNullable(field.getFieldName(), field.getDataType());
+    }
+    return builder.buildSchema();
+  }
+
+
+  private boolean parseLine(String line, RowSetLoader writer) throws IOException {
+    int i = 0;
+    TypeProtos.MinorType dataType;
+    String dateTimeFormat;
+    String value;
+    for (FixedwidthFieldConfig field : config.getFields()) {
+      value = line.substring(field.getStartIndex() - 1, field.getStartIndex() + field.getFieldWidth() - 1);
+      dataType = field.getDataType();
+      dateTimeFormat = field.getDateTimeFormat();
+      DateTimeFormatter formatter = DateTimeFormatter.ofPattern(dateTimeFormat, Locale.ENGLISH);
+      try {
+        switch (dataType) {

Review comment:
       This is OK, but slow because of the switch. There is a set of field converter classes which can handle the string-to-whatever conversions. With that, there is a direct call per field (inner loop) from reading the field to convert to write into value vectors. The field-specific-type switch is done only once, at setup time.
   
   These converters are used in the CSV reader when a schema is provided. I can dig up more examples if helpful.




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

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

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



[GitHub] [drill] dzamo commented on pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
dzamo commented on pull request #2282:
URL: https://github.com/apache/drill/pull/2282#issuecomment-962939201


   @MFoss19 @estherbuchwalter following some [recent chat](https://github.com/apache/drill/pull/2359#issuecomment-962673076) with @paul-rogers and my last comment here, how about a reduced format config such as the following?
   
   ```json
   "fixedwidth": {
     "type": "fixedwidth",
     "extensions": [
       "fwf"
     ],
     "extractHeader": true,
     "trimStrings": true,
     "columnOffsets": [1, 11, 21, 31],
     "columnWidths": [10, 10, 10, 10]
   }
   ```
   
   Column names and types can already come from a provided schema or aliasing after calls to `CAST()`.  Incidentally, the settings above can be overriden per query using a provided schema too.


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

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

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



[GitHub] [drill] MFoss19 commented on a change in pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
MFoss19 commented on a change in pull request #2282:
URL: https://github.com/apache/drill/pull/2282#discussion_r707560617



##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthFieldConfig.java
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.fixedwidth;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.common.types.TypeProtos;
+
+@JsonTypeName("fixedwidthReaderFieldDescription")
+@JsonInclude(JsonInclude.Include.NON_DEFAULT)
+public class FixedwidthFieldConfig {
+
+  private final TypeProtos.MinorType dataType;
+  private final String fieldName;
+  private final String dateTimeFormat;
+  private final int startIndex;
+  private final int fieldWidth;
+
+  public FixedwidthFieldConfig(@JsonProperty("dataType") TypeProtos.MinorType dataType,

Review comment:
       This was tested configuring the plugin within the Drill UI (via manual test). I will also add an automated unit test for parsing the Json. To answer your question, yes it works to use MinorType here. Jackson can always read in Java enums, MinorType was generated as part of our protobuf and for this type of data they generate proper Java enums.




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

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

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



[GitHub] [drill] paul-rogers commented on pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
paul-rogers commented on pull request #2282:
URL: https://github.com/apache/drill/pull/2282#issuecomment-995956782


   @jnturton, turns out the required changes are sitting in a branch in my private repo, `csv`, that somehow never not converted to a PR. I'll see if I can merge that stuff into a PR. 


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

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

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



[GitHub] [drill] lgtm-com[bot] commented on pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #2282:
URL: https://github.com/apache/drill/pull/2282#issuecomment-978080469


   This pull request **introduces 2 alerts** when merging 881d4658e62c106c13364bf1b4b10e66d3f013f7 into 38d0c1d9586b7adc8adfb89f574085cdc26595fe - [view on LGTM.com](https://lgtm.com/projects/g/apache/drill/rev/pr-bd64888103cc1e954b03c39b501591e29f641125)
   
   **new alerts:**
   
   * 2 for Unused format argument


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

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

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



[GitHub] [drill] MFoss19 commented on a change in pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
MFoss19 commented on a change in pull request #2282:
URL: https://github.com/apache/drill/pull/2282#discussion_r681093297



##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthFieldConfig.java
##########
@@ -0,0 +1,71 @@
+package org.apache.drill.exec.store.fixedwidth;

Review comment:
       Done




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

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

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



[GitHub] [drill] MFoss19 commented on a change in pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
MFoss19 commented on a change in pull request #2282:
URL: https://github.com/apache/drill/pull/2282#discussion_r684483938



##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator>{
+
+  private static final Logger logger = LoggerFactory.getLogger(FixedwidthBatchReader.class);
+
+  private FileSplit split;
+
+  private final int maxRecords;
+
+  private final FixedwidthFormatConfig config;
+
+  private CustomErrorContext errorContext;
+
+  private InputStream fsStream;
+
+  private ResultSetLoader loader;
+
+  private BufferedReader reader;
+
+  public FixedwidthBatchReader(FixedwidthFormatConfig config, int maxRecords) {
+    this.config = config;
+    this.maxRecords = maxRecords;
+  }
+
+  @Override
+  public boolean open(FileSchemaNegotiator negotiator) {
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      negotiator.tableSchema(buildSchema(),true);
+      loader = negotiator.build();
+    } catch (Exception e) {
+      throw UserException
+              .dataReadError(e)
+              .message("Failed to open input file: {}", split.getPath().toString())
+              .addContext(errorContext)
+              .addContext(e.getMessage())
+              .build(logger);
+    }
+
+    reader = new BufferedReader(new InputStreamReader(fsStream, Charsets.UTF_8));
+
+    return true;
+
+  }
+
+  @Override
+  public boolean next() { // Use loader to read data from file to turn into Drill rows
+

Review comment:
       Fixed use of MaxRecords in next() function.
   
   Still looking at moving writer start and end to next.




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

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

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



[GitHub] [drill] cgivre commented on a change in pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
cgivre commented on a change in pull request #2282:
URL: https://github.com/apache/drill/pull/2282#discussion_r756495900



##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.math.BigDecimal;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(FixedwidthBatchReader.class);
+  private FileSplit split;
+  private final int maxRecords;
+  private final FixedwidthFormatConfig config;
+  private CustomErrorContext errorContext;
+  private InputStream fsStream;
+  private ResultSetLoader loader;
+  private RowSetLoader writer;
+  private BufferedReader reader;
+  private int lineNum;
+
+  public FixedwidthBatchReader(FixedwidthFormatConfig config, int maxRecords) {
+    this.config = config;
+    this.maxRecords = maxRecords;
+  }
+
+  @Override
+  public boolean open(FileSchemaNegotiator negotiator) {
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    lineNum = 0;
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      negotiator.tableSchema(buildSchema(), true);
+      loader = negotiator.build();
+    } catch (Exception e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to open input file: {}", split.getPath().toString())
+        .addContext(errorContext)
+        .addContext(e.getMessage())
+        .build(logger);
+    }
+    reader = new BufferedReader(new InputStreamReader(fsStream, Charsets.UTF_8));
+    return true;
+  }
+
+  @Override
+  public boolean next() { // Use loader to read data from file to turn into Drill rows
+    String line;
+    RowSetLoader writer = loader.writer();

Review comment:
       This line should be in the `open()` method. 

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.math.BigDecimal;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(FixedwidthBatchReader.class);
+  private FileSplit split;
+  private final int maxRecords;
+  private final FixedwidthFormatConfig config;
+  private CustomErrorContext errorContext;
+  private InputStream fsStream;
+  private ResultSetLoader loader;
+  private RowSetLoader writer;
+  private BufferedReader reader;
+  private int lineNum;
+
+  public FixedwidthBatchReader(FixedwidthFormatConfig config, int maxRecords) {
+    this.config = config;
+    this.maxRecords = maxRecords;
+  }
+
+  @Override
+  public boolean open(FileSchemaNegotiator negotiator) {
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    lineNum = 0;
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      negotiator.tableSchema(buildSchema(), true);
+      loader = negotiator.build();
+    } catch (Exception e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to open input file: {}", split.getPath().toString())
+        .addContext(errorContext)
+        .addContext(e.getMessage())
+        .build(logger);
+    }
+    reader = new BufferedReader(new InputStreamReader(fsStream, Charsets.UTF_8));
+    return true;
+  }
+
+  @Override
+  public boolean next() { // Use loader to read data from file to turn into Drill rows
+    String line;
+    RowSetLoader writer = loader.writer();
+
+    try {
+      line = reader.readLine();

Review comment:
       Why not include this in the loop?

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthFormatPlugin.java
##########
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.drill.exec.store.fixedwidth;
+
+import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileReaderFactory;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileScanBuilder;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.options.OptionManager;
+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 FixedwidthFormatPlugin extends EasyFormatPlugin<FixedwidthFormatConfig> {
+
+  protected static final String DEFAULT_NAME = "fixedwidth";
+
+  private static class FixedwidthReaderFactory extends FileReaderFactory {
+
+    private final FixedwidthFormatConfig config;
+    private final int maxRecords;
+
+    public FixedwidthReaderFactory(FixedwidthFormatConfig config, int maxRecords) {
+      this.config = config;
+      this.maxRecords = maxRecords;
+    }
+
+    @Override
+    public ManagedReader<? extends FileSchemaNegotiator> newReader() {
+      return new FixedwidthBatchReader(config, maxRecords);
+    }
+  }
+
+  public FixedwidthFormatPlugin(String name,
+                                DrillbitContext context,
+                                Configuration fsConf,
+                                StoragePluginConfig storageConfig,
+                                FixedwidthFormatConfig formatConfig) {
+    super(name, easyConfig(fsConf, formatConfig), context, storageConfig, formatConfig);
+  }
+
+  private static EasyFormatConfig easyConfig(Configuration fsConf, FixedwidthFormatConfig pluginConfig) {
+    return EasyFormatConfig.builder()
+      .readable(true)
+      .writable(false)
+      .blockSplittable(false) // Change to true

Review comment:
       Change to true. 

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.math.BigDecimal;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(FixedwidthBatchReader.class);
+  private FileSplit split;
+  private final int maxRecords;
+  private final FixedwidthFormatConfig config;
+  private CustomErrorContext errorContext;
+  private InputStream fsStream;
+  private ResultSetLoader loader;
+  private RowSetLoader writer;
+  private BufferedReader reader;
+  private int lineNum;
+
+  public FixedwidthBatchReader(FixedwidthFormatConfig config, int maxRecords) {
+    this.config = config;
+    this.maxRecords = maxRecords;
+  }
+
+  @Override
+  public boolean open(FileSchemaNegotiator negotiator) {
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    lineNum = 0;
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      negotiator.tableSchema(buildSchema(), true);

Review comment:
       Here's where you can check to see whether the user provided a schema or not.   You could do something like this:
   
   ```java
      if (negotiator.hasProvidedSchema()) {
         TupleMetadata providedSchema = negotiator.providedSchema();
         // Build column writer array
         negotiator.tableSchema(finalSchema, true);
       } else {
         negotiator.tableSchema(buildSchema(), true);
       }
   ```

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.math.BigDecimal;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(FixedwidthBatchReader.class);
+  private FileSplit split;
+  private final int maxRecords;
+  private final FixedwidthFormatConfig config;
+  private CustomErrorContext errorContext;
+  private InputStream fsStream;
+  private ResultSetLoader loader;
+  private RowSetLoader writer;
+  private BufferedReader reader;
+  private int lineNum;
+
+  public FixedwidthBatchReader(FixedwidthFormatConfig config, int maxRecords) {
+    this.config = config;
+    this.maxRecords = maxRecords;
+  }
+
+  @Override
+  public boolean open(FileSchemaNegotiator negotiator) {
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    lineNum = 0;
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      negotiator.tableSchema(buildSchema(), true);
+      loader = negotiator.build();
+    } catch (Exception e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to open input file: {}", split.getPath().toString())
+        .addContext(errorContext)
+        .addContext(e.getMessage())

Review comment:
       You can remove this second line.   Also, please add `e.getMessage()` to the message line. 

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.math.BigDecimal;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(FixedwidthBatchReader.class);
+  private FileSplit split;
+  private final int maxRecords;
+  private final FixedwidthFormatConfig config;
+  private CustomErrorContext errorContext;
+  private InputStream fsStream;
+  private ResultSetLoader loader;
+  private RowSetLoader writer;
+  private BufferedReader reader;
+  private int lineNum;
+
+  public FixedwidthBatchReader(FixedwidthFormatConfig config, int maxRecords) {
+    this.config = config;
+    this.maxRecords = maxRecords;
+  }
+
+  @Override
+  public boolean open(FileSchemaNegotiator negotiator) {
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    lineNum = 0;
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      negotiator.tableSchema(buildSchema(), true);
+      loader = negotiator.build();
+    } catch (Exception e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to open input file: {}", split.getPath().toString())
+        .addContext(errorContext)
+        .addContext(e.getMessage())
+        .build(logger);
+    }
+    reader = new BufferedReader(new InputStreamReader(fsStream, Charsets.UTF_8));
+    return true;
+  }
+
+  @Override
+  public boolean next() { // Use loader to read data from file to turn into Drill rows
+    String line;
+    RowSetLoader writer = loader.writer();
+
+    try {
+      line = reader.readLine();
+      while (!writer.isFull() && line != null) {
+        writer.start();
+        parseLine(line, writer);
+        writer.save();
+        line = reader.readLine();
+        lineNum++;
+      }
+    } catch (IOException e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to read input file: {}", split.getPath().toString())
+        .addContext(errorContext)
+        .addContext(e.getMessage())

Review comment:
       For the error message, you don't need to have multiple `addContext()` calls.  The main thing is to pass the `errorContext`.  I would add the `e.getMessage()` to the `message()` call. 

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.math.BigDecimal;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(FixedwidthBatchReader.class);
+  private FileSplit split;
+  private final int maxRecords;
+  private final FixedwidthFormatConfig config;
+  private CustomErrorContext errorContext;
+  private InputStream fsStream;
+  private ResultSetLoader loader;
+  private RowSetLoader writer;
+  private BufferedReader reader;
+  private int lineNum;
+
+  public FixedwidthBatchReader(FixedwidthFormatConfig config, int maxRecords) {
+    this.config = config;
+    this.maxRecords = maxRecords;
+  }
+
+  @Override
+  public boolean open(FileSchemaNegotiator negotiator) {
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    lineNum = 0;
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      negotiator.tableSchema(buildSchema(), true);
+      loader = negotiator.build();
+    } catch (Exception e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to open input file: {}", split.getPath().toString())
+        .addContext(errorContext)
+        .addContext(e.getMessage())
+        .build(logger);
+    }
+    reader = new BufferedReader(new InputStreamReader(fsStream, Charsets.UTF_8));
+    return true;
+  }
+
+  @Override
+  public boolean next() { // Use loader to read data from file to turn into Drill rows
+    String line;
+    RowSetLoader writer = loader.writer();
+
+    try {
+      line = reader.readLine();
+      while (!writer.isFull() && line != null) {
+        writer.start();
+        parseLine(line, writer);
+        writer.save();
+        line = reader.readLine();
+        lineNum++;
+      }
+    } catch (IOException e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to read input file: {}", split.getPath().toString())
+        .addContext(errorContext)
+        .addContext(e.getMessage())

Review comment:
       Here and elsewhere.

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.math.BigDecimal;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(FixedwidthBatchReader.class);
+  private FileSplit split;
+  private final int maxRecords;
+  private final FixedwidthFormatConfig config;
+  private CustomErrorContext errorContext;
+  private InputStream fsStream;
+  private ResultSetLoader loader;
+  private RowSetLoader writer;
+  private BufferedReader reader;
+  private int lineNum;
+
+  public FixedwidthBatchReader(FixedwidthFormatConfig config, int maxRecords) {
+    this.config = config;
+    this.maxRecords = maxRecords;
+  }
+
+  @Override
+  public boolean open(FileSchemaNegotiator negotiator) {
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    lineNum = 0;
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      negotiator.tableSchema(buildSchema(), true);
+      loader = negotiator.build();
+    } catch (Exception e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to open input file: {}", split.getPath().toString())
+        .addContext(errorContext)
+        .addContext(e.getMessage())
+        .build(logger);
+    }
+    reader = new BufferedReader(new InputStreamReader(fsStream, Charsets.UTF_8));
+    return true;
+  }
+
+  @Override
+  public boolean next() { // Use loader to read data from file to turn into Drill rows
+    String line;
+    RowSetLoader writer = loader.writer();
+
+    try {
+      line = reader.readLine();
+      while (!writer.isFull() && line != null) {
+        writer.start();
+        parseLine(line, writer);
+        writer.save();
+        line = reader.readLine();
+        lineNum++;
+      }
+    } catch (IOException e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to read input file: {}", split.getPath().toString())
+        .addContext(errorContext)
+        .addContext(e.getMessage())
+        .addContext("Line Number", lineNum)
+        .build(logger);
+    }
+    return writer.limitReached(maxRecords);  // returns false when maxRecords limit has been reached

Review comment:
       The `next()` method needs some work.   Really this should be called `nextBatch()` as the next method returns `true` when there is more data, to read, `false` if not. 
   
   ```java 
     @Override
     public boolean next() {
       while (!rowWriter.isFull()) {
         if (!processNextLine()) {
           return false;
         }
       }
       return true;
     }
   ```
   
   This method will iterate through the batch of data, and when the `rowWriter` is full, (IE the batch is full) it will stop reading, BUT the method will return `true` because there is more data to read.  The limit is pushed down in the `processNextLine()` method. 
   
   

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.math.BigDecimal;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(FixedwidthBatchReader.class);
+  private FileSplit split;
+  private final int maxRecords;
+  private final FixedwidthFormatConfig config;
+  private CustomErrorContext errorContext;
+  private InputStream fsStream;
+  private ResultSetLoader loader;
+  private RowSetLoader writer;
+  private BufferedReader reader;
+  private int lineNum;
+
+  public FixedwidthBatchReader(FixedwidthFormatConfig config, int maxRecords) {
+    this.config = config;
+    this.maxRecords = maxRecords;
+  }
+
+  @Override
+  public boolean open(FileSchemaNegotiator negotiator) {
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    lineNum = 0;
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      negotiator.tableSchema(buildSchema(), true);
+      loader = negotiator.build();
+    } catch (Exception e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to open input file: {}", split.getPath().toString())
+        .addContext(errorContext)
+        .addContext(e.getMessage())
+        .build(logger);
+    }
+    reader = new BufferedReader(new InputStreamReader(fsStream, Charsets.UTF_8));
+    return true;
+  }
+
+  @Override
+  public boolean next() { // Use loader to read data from file to turn into Drill rows
+    String line;
+    RowSetLoader writer = loader.writer();
+
+    try {
+      line = reader.readLine();
+      while (!writer.isFull() && line != null) {
+        writer.start();
+        parseLine(line, writer);
+        writer.save();
+        line = reader.readLine();
+        lineNum++;
+      }
+    } catch (IOException e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to read input file: {}", split.getPath().toString())
+        .addContext(errorContext)
+        .addContext(e.getMessage())
+        .addContext("Line Number", lineNum)
+        .build(logger);
+    }
+    return writer.limitReached(maxRecords);  // returns false when maxRecords limit has been reached
+  }
+
+  @Override
+  public void close() {
+    if (fsStream != null){
+      AutoCloseables.closeSilently(fsStream);

Review comment:
       This line should be out of the if statement.

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,200 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.math.BigDecimal;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(FixedwidthBatchReader.class);
+  private FileSplit split;
+  private final int maxRecords;
+  private final FixedwidthFormatConfig config;
+  private CustomErrorContext errorContext;
+  private InputStream fsStream;
+  private ResultSetLoader loader;
+  private RowSetLoader writer;
+  private BufferedReader reader;
+  private int lineNum;
+
+  public FixedwidthBatchReader(FixedwidthFormatConfig config, int maxRecords) {
+    this.config = config;
+    this.maxRecords = maxRecords;
+  }
+
+
+  @Override
+  public boolean open(FileSchemaNegotiator negotiator) {
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    lineNum = 0;
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      negotiator.tableSchema(buildSchema(), true);
+      loader = negotiator.build();
+    } catch (Exception e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to open input file: {}", split.getPath().toString())
+        .addContext(errorContext)
+        .addContext(e.getMessage())
+        .build(logger);
+    }
+    reader = new BufferedReader(new InputStreamReader(fsStream, Charsets.UTF_8));
+    return true;
+  }
+
+  @Override
+  public boolean next() { // Use loader to read data from file to turn into Drill rows
+    String line;
+    RowSetLoader writer = loader.writer();
+
+    try {
+      line = reader.readLine();
+      while (!writer.isFull() && line != null) {
+        writer.start();
+        parseLine(line, writer);
+        writer.save();
+        line = reader.readLine();
+        lineNum++;
+      }
+    } catch (IOException e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to read input file: {}", split.getPath().toString())
+        .addContext(errorContext)
+        .addContext(e.getMessage())
+        .addContext("Line Number", lineNum)
+        .build(logger);
+    }
+    return writer.limitReached(maxRecords);  // returns false when maxRecords limit has been reached
+  }
+
+  @Override
+  public void close() {
+    if (fsStream != null){
+      AutoCloseables.closeSilently(fsStream);
+      fsStream = null;
+    }
+  }
+
+  private TupleMetadata buildSchema() {
+    SchemaBuilder builder = new SchemaBuilder();
+    for (FixedwidthFieldConfig field : config.getFields()) {
+      if (field.getType() == TypeProtos.MinorType.VARDECIMAL){
+        builder.addNullable(field.getName(), TypeProtos.MinorType.VARDECIMAL,38,4);
+        //revisit this
+      } else {
+        builder.addNullable(field.getName(), field.getType());
+      }
+    }
+    return builder.buildSchema();
+  }
+
+
+  private boolean parseLine(String line, RowSetLoader writer) throws IOException {
+    int i = 0;
+    TypeProtos.MinorType dataType;
+    String dateTimeFormat;
+    String value;
+    for (FixedwidthFieldConfig field : config.getFields()) {

Review comment:
       My original understanding of this was that for the fixed width plugin was that it would work in a similar manner to the log regex reader where the user provides the schema in the config, either in the format config or at query time using the `table()` function. 




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

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

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



[GitHub] [drill] lgtm-com[bot] commented on pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #2282:
URL: https://github.com/apache/drill/pull/2282#issuecomment-1075602428


   This pull request **introduces 1 alert** when merging bf6a16c3b0e5ccb41b4356e284135e8358baa25f into 4e97f5c7d9875dfda5785c6f01bc137973183d2b - [view on LGTM.com](https://lgtm.com/projects/g/apache/drill/rev/pr-de21eddbbc4e31ec579b02bc3531b521202edea1)
   
   **new alerts:**
   
   * 1 for Unused format argument


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

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

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



[GitHub] [drill] cgivre commented on a change in pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
cgivre commented on a change in pull request #2282:
URL: https://github.com/apache/drill/pull/2282#discussion_r740503925



##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator> {

Review comment:
       @paul-rogers Is there an example somewhere of how to use the rowset loader to set up the schema automatically?  Is this as simple as checking to see whether the schema is provided and if so, use that?




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

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

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



[GitHub] [drill] lgtm-com[bot] commented on pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #2282:
URL: https://github.com/apache/drill/pull/2282#issuecomment-962044746


   This pull request **introduces 2 alerts** when merging 9b95c45f7f5ddbb7543c89a1efac99bda0ebff8c into 52838ef26e5e3e6b4461c2c656ffada0c64c9e88 - [view on LGTM.com](https://lgtm.com/projects/g/apache/drill/rev/pr-7d8671f78491ffc2026ea94d10c597de912e7eec)
   
   **new alerts:**
   
   * 2 for Unused format argument


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

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

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



[GitHub] [drill] cgivre commented on a change in pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
cgivre commented on a change in pull request #2282:
URL: https://github.com/apache/drill/pull/2282#discussion_r679426218



##########
File path: contrib/format-fixedwidth/pom.xml
##########
@@ -0,0 +1,84 @@
+<?xml version="1.0"?>
+<!--
+
+    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.
+
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">

Review comment:
       Nit:  Can you please reformat with 2 space indentation. 

##########
File path: contrib/format-fixedwidth/src/test/java/org/apache/drill/exec/store/fixedwidth/TestFixedwidthRecordReader.java
##########
@@ -0,0 +1,126 @@
+package org.apache.drill.exec.store.fixedwidth;
+
+import org.apache.drill.categories.RowSetTests;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.rowSet.RowSet;
+import org.apache.drill.exec.physical.rowSet.RowSetBuilder;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.drill.test.ClusterFixture;
+import org.apache.drill.test.ClusterTest;
+import org.apache.drill.test.QueryBuilder;
+import org.apache.drill.test.rowSet.RowSetComparison;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.nio.file.Paths;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+
+import static org.junit.Assert.assertEquals;
+
+@Category(RowSetTests.class)
+public class TestFixedwidthRecordReader extends ClusterTest {
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    ClusterTest.startCluster(ClusterFixture.builder(dirTestWatcher));
+
+    FixedwidthFormatConfig formatConfig = new FixedwidthFormatConfig(Lists.newArrayList("fwf"),
+            Lists.newArrayList(
+            new FixedwidthFieldConfig(TypeProtos.MinorType.INT, "Number", "", 1, 4),
+            new FixedwidthFieldConfig(TypeProtos.MinorType.VARCHAR, "Letter", "", 6, 4),
+            new FixedwidthFieldConfig(TypeProtos.MinorType.INT,"Address","",11,3),
+            new FixedwidthFieldConfig(TypeProtos.MinorType.DATE,"Date","MM-dd-yyyy",15,10),
+            new FixedwidthFieldConfig(TypeProtos.MinorType.TIME,"Time","HH:mm:ss",26,8),
+            new FixedwidthFieldConfig(TypeProtos.MinorType.TIMESTAMP,"DateTime","MM-dd-yyyy'T'HH:mm:ss.SSX",35,23)
+    ));
+    cluster.defineFormat("cp", "fwf", formatConfig);
+
+    // Needed for compressed file unit test
+    dirTestWatcher.copyResourceToRoot(Paths.get("fwf/"));
+  }
+
+  @Test
+  public void testExplicitQuery() throws Exception {
+    String sql = "SELECT ID, Urban, Urban_value FROM dfs.`spss/testdata.sav` WHERE d16=4";
+
+    QueryBuilder q = client.queryBuilder().sql(sql);
+    RowSet results = q.rowSet();
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+            .addNullable("ID", TypeProtos.MinorType.FLOAT8)
+            .addNullable("Urban", TypeProtos.MinorType.FLOAT8)
+            .addNullable("Urban_value", TypeProtos.MinorType.VARCHAR)
+            .buildSchema();
+
+
+    RowSet expected = new RowSetBuilder(client.allocator(), expectedSchema)
+            .addRow(47.0, 1.0, "Urban").addRow(53.0, 1.0, "Urban")
+            .addRow(66.0, 1.0, "Urban")
+            .build();
+
+    assertEquals(3, results.rowCount());
+
+    new RowSetComparison(expected).verifyAndClearAll(results);
+  }
+
+  @Test
+  public void testBatchReader() throws Exception {
+    String sql = "SELECT * FROM cp.`fwf/test.fwf`";
+    RowSet results = client.queryBuilder().sql(sql).rowSet();
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+            .addNullable("Number", TypeProtos.MinorType.INT)
+            .addNullable("Letter", TypeProtos.MinorType.VARCHAR)
+            .addNullable("Address", TypeProtos.MinorType.INT)
+            .addNullable("Date", TypeProtos.MinorType.DATE)
+            .addNullable("Time",TypeProtos.MinorType.TIME)
+            .addNullable("DateTime",TypeProtos.MinorType.TIMESTAMP)
+            .buildSchema();
+
+
+    RowSet expected = new RowSetBuilder(client.allocator(), expectedSchema)
+            .addRow(1234, "test", 567, LocalDate.parse("2021-02-10"), LocalTime.parse("10:30:27"), Instant.parse("2021-02-10T15:30:27.00Z"))
+            .addRow(5678, "TEST", 890, LocalDate.parse("2021-07-27"), LocalTime.parse("12:40:15"), Instant.parse("2021-07-27T16:40:15.00Z"))
+            .addRow(1111, "abcd", 111, LocalDate.parse("1111-11-11"), LocalTime.parse("11:11:11"), Instant.parse("1111-11-11T16:28:43.11Z"))
+            .addRow(2222, "efgh", 222, LocalDate.parse("2222-01-22"), LocalTime.parse("22:22:22"), Instant.parse("2222-01-23T03:22:22.22Z"))
+            .addRow(3333, "ijkl", 333, LocalDate.parse("3333-02-01"), LocalTime.parse("01:33:33"), Instant.parse("3333-02-01T06:33:33.33Z"))
+            .addRow(4444, "mnop", 444, LocalDate.parse("4444-03-02"), LocalTime.parse("02:44:44"), Instant.parse("4444-03-02T07:44:44.44Z"))
+            .addRow(5555, "qrst", 555, LocalDate.parse("5555-04-03"), LocalTime.parse("03:55:55"), Instant.parse("5555-04-03T07:55:55.55Z"))
+            .addRow(6666, "uvwx", 666, LocalDate.parse("6666-05-04"), LocalTime.parse("04:01:01"), Instant.parse("6666-05-04T08:01:01.01Z"))
+            .addRow(7777, "yzzz", 777, LocalDate.parse("7777-06-05"), LocalTime.parse("05:11:11"), Instant.parse("7777-06-05T09:11:11.11Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .build();
+
+    System.out.println(expected);
+    assertEquals(25, results.rowCount());
+
+    //System.out.println(results.batchSchema());

Review comment:
       Please remove any calls to `System.out.println` in unit tests. 

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator>{
+
+  private static final Logger logger = LoggerFactory.getLogger(FixedwidthBatchReader.class);
+
+  private FileSplit split;
+
+  private final int maxRecords;
+
+  private final FixedwidthFormatConfig config;
+
+  private CustomErrorContext errorContext;
+
+  private InputStream fsStream;
+
+  private ResultSetLoader loader;
+
+  private BufferedReader reader;
+
+  public FixedwidthBatchReader(FixedwidthFormatConfig config, int maxRecords) {
+    this.config = config;
+    this.maxRecords = maxRecords;
+  }
+
+  @Override
+  public boolean open(FileSchemaNegotiator negotiator) {
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      negotiator.tableSchema(buildSchema(),true);
+      loader = negotiator.build();
+    } catch (Exception e) {
+      throw UserException
+              .dataReadError(e)
+              .message("Failed to open input file: {}", split.getPath().toString())
+              .addContext(errorContext)
+              .addContext(e.getMessage())
+              .build(logger);
+    }
+
+    reader = new BufferedReader(new InputStreamReader(fsStream, Charsets.UTF_8));
+
+    return true;
+
+  }
+
+  @Override
+  public boolean next() { // Use loader to read data from file to turn into Drill rows
+
+    String line;
+
+    try {
+      line = reader.readLine();
+      RowSetLoader writer = loader.writer();
+
+      while (!writer.isFull() && line != null) {
+
+        writer.start();
+        parseLine(line, writer);
+        writer.save();
+
+        line = reader.readLine();
+      }
+    } catch (Exception e) {
+      throw UserException
+              .dataReadError(e)
+              .message("Failed to read input file: {}", split.getPath().toString())
+              .addContext(errorContext)
+              .addContext(e.getMessage())
+              .build(logger);
+    }
+    return (line != null);
+  }
+
+  @Override
+  public void close() {
+    try {
+      fsStream.close();
+      loader.close();
+    } catch (Exception e) {
+      throw UserException
+              .dataReadError(e)
+              .message("Failed to close input file: {}", split.getPath().toString())
+              .addContext(errorContext)
+              .addContext(e.getMessage())
+              .build(logger);
+    }
+  }
+
+  private TupleMetadata buildSchema(){
+    SchemaBuilder builder = new SchemaBuilder();
+
+    for (FixedwidthFieldConfig field : config.getFields()){
+      builder.addNullable(field.getFieldName(),field.getDataType());
+    }
+
+      return builder.buildSchema();
+  }
+
+  private void parseLine(String line, RowSetLoader writer) {
+    int i = 0;
+    TypeProtos.MinorType dataType;
+    String dateTimeFormat;
+    String value;
+
+    for (FixedwidthFieldConfig field : config.getFields()) {
+      value = line.substring(field.getStartIndex() - 1, field.getStartIndex() + field.getFieldWidth() - 1);
+
+      dataType = field.getDataType();
+      dateTimeFormat = field.getDateTimeFormat();
+      DateTimeFormatter formatter = DateTimeFormatter.ofPattern(dateTimeFormat, Locale.ENGLISH);
+
+      switch (dataType) {
+        case INT:
+          writer.scalar(i).setInt(Integer.parseInt(value));
+          break;
+        case VARCHAR:
+          writer.scalar(i).setString(value);
+          break;
+        case DATE:
+          LocalDate date = LocalDate.parse(value, formatter);
+          writer.scalar(i).setDate(date);
+          break;
+        case TIME:
+          LocalTime time = LocalTime.parse(value, formatter);
+          writer.scalar(i).setTime(time);
+          break;
+        case TIMESTAMP:
+          LocalDateTime ldt = LocalDateTime.parse(value,formatter);
+          ZoneId z = ZoneId.of( "America/Toronto" );
+          ZonedDateTime zdt = ldt.atZone( z );
+          Instant timeStamp = zdt.toInstant();
+          writer.scalar(i).setTimestamp(timeStamp);
+          break;
+        default:
+          throw new RuntimeException("Unknown data type specified in fixed width. Found data type " + dataType);
+

Review comment:
       Nit: Please remove excess whitespace here and elsewhere.

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator>{
+
+  private static final Logger logger = LoggerFactory.getLogger(FixedwidthBatchReader.class);
+
+  private FileSplit split;
+
+  private final int maxRecords;
+
+  private final FixedwidthFormatConfig config;
+
+  private CustomErrorContext errorContext;
+
+  private InputStream fsStream;
+
+  private ResultSetLoader loader;
+
+  private BufferedReader reader;
+
+  public FixedwidthBatchReader(FixedwidthFormatConfig config, int maxRecords) {
+    this.config = config;
+    this.maxRecords = maxRecords;
+  }
+
+  @Override
+  public boolean open(FileSchemaNegotiator negotiator) {
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      negotiator.tableSchema(buildSchema(),true);
+      loader = negotiator.build();
+    } catch (Exception e) {
+      throw UserException
+              .dataReadError(e)
+              .message("Failed to open input file: {}", split.getPath().toString())
+              .addContext(errorContext)
+              .addContext(e.getMessage())
+              .build(logger);
+    }
+
+    reader = new BufferedReader(new InputStreamReader(fsStream, Charsets.UTF_8));
+
+    return true;
+
+  }
+
+  @Override
+  public boolean next() { // Use loader to read data from file to turn into Drill rows
+
+    String line;
+
+    try {
+      line = reader.readLine();
+      RowSetLoader writer = loader.writer();
+
+      while (!writer.isFull() && line != null) {
+
+        writer.start();
+        parseLine(line, writer);
+        writer.save();
+
+        line = reader.readLine();
+      }
+    } catch (Exception e) {
+      throw UserException
+              .dataReadError(e)
+              .message("Failed to read input file: {}", split.getPath().toString())
+              .addContext(errorContext)
+              .addContext(e.getMessage())
+              .build(logger);
+    }
+    return (line != null);
+  }
+
+  @Override
+  public void close() {
+    try {
+      fsStream.close();
+      loader.close();
+    } catch (Exception e) {
+      throw UserException
+              .dataReadError(e)
+              .message("Failed to close input file: {}", split.getPath().toString())
+              .addContext(errorContext)
+              .addContext(e.getMessage())
+              .build(logger);
+    }
+  }
+
+  private TupleMetadata buildSchema(){
+    SchemaBuilder builder = new SchemaBuilder();
+
+    for (FixedwidthFieldConfig field : config.getFields()){
+      builder.addNullable(field.getFieldName(),field.getDataType());
+    }
+
+      return builder.buildSchema();
+  }
+
+  private void parseLine(String line, RowSetLoader writer) {
+    int i = 0;
+    TypeProtos.MinorType dataType;
+    String dateTimeFormat;
+    String value;
+
+    for (FixedwidthFieldConfig field : config.getFields()) {
+      value = line.substring(field.getStartIndex() - 1, field.getStartIndex() + field.getFieldWidth() - 1);
+
+      dataType = field.getDataType();
+      dateTimeFormat = field.getDateTimeFormat();
+      DateTimeFormatter formatter = DateTimeFormatter.ofPattern(dateTimeFormat, Locale.ENGLISH);
+
+      switch (dataType) {
+        case INT:

Review comment:
       It looks like we are only supporting INT, VARCHAR, DATE, TIME and TIMESTAMP.  Do we want to support a few other data types such as `LONG` or `DOUBLE`?

##########
File path: contrib/format-fixedwidth/src/test/java/org/apache/drill/exec/store/fixedwidth/TestFixedwidthRecordReader.java
##########
@@ -0,0 +1,126 @@
+package org.apache.drill.exec.store.fixedwidth;
+
+import org.apache.drill.categories.RowSetTests;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.rowSet.RowSet;
+import org.apache.drill.exec.physical.rowSet.RowSetBuilder;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.drill.test.ClusterFixture;
+import org.apache.drill.test.ClusterTest;
+import org.apache.drill.test.QueryBuilder;
+import org.apache.drill.test.rowSet.RowSetComparison;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.nio.file.Paths;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+
+import static org.junit.Assert.assertEquals;
+
+@Category(RowSetTests.class)
+public class TestFixedwidthRecordReader extends ClusterTest {
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    ClusterTest.startCluster(ClusterFixture.builder(dirTestWatcher));
+
+    FixedwidthFormatConfig formatConfig = new FixedwidthFormatConfig(Lists.newArrayList("fwf"),
+            Lists.newArrayList(
+            new FixedwidthFieldConfig(TypeProtos.MinorType.INT, "Number", "", 1, 4),
+            new FixedwidthFieldConfig(TypeProtos.MinorType.VARCHAR, "Letter", "", 6, 4),
+            new FixedwidthFieldConfig(TypeProtos.MinorType.INT,"Address","",11,3),
+            new FixedwidthFieldConfig(TypeProtos.MinorType.DATE,"Date","MM-dd-yyyy",15,10),
+            new FixedwidthFieldConfig(TypeProtos.MinorType.TIME,"Time","HH:mm:ss",26,8),
+            new FixedwidthFieldConfig(TypeProtos.MinorType.TIMESTAMP,"DateTime","MM-dd-yyyy'T'HH:mm:ss.SSX",35,23)
+    ));
+    cluster.defineFormat("cp", "fwf", formatConfig);
+
+    // Needed for compressed file unit test
+    dirTestWatcher.copyResourceToRoot(Paths.get("fwf/"));
+  }
+
+  @Test
+  public void testExplicitQuery() throws Exception {
+    String sql = "SELECT ID, Urban, Urban_value FROM dfs.`spss/testdata.sav` WHERE d16=4";
+
+    QueryBuilder q = client.queryBuilder().sql(sql);
+    RowSet results = q.rowSet();
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+            .addNullable("ID", TypeProtos.MinorType.FLOAT8)
+            .addNullable("Urban", TypeProtos.MinorType.FLOAT8)
+            .addNullable("Urban_value", TypeProtos.MinorType.VARCHAR)
+            .buildSchema();
+
+
+    RowSet expected = new RowSetBuilder(client.allocator(), expectedSchema)
+            .addRow(47.0, 1.0, "Urban").addRow(53.0, 1.0, "Urban")
+            .addRow(66.0, 1.0, "Urban")
+            .build();
+
+    assertEquals(3, results.rowCount());
+
+    new RowSetComparison(expected).verifyAndClearAll(results);
+  }
+
+  @Test
+  public void testBatchReader() throws Exception {
+    String sql = "SELECT * FROM cp.`fwf/test.fwf`";
+    RowSet results = client.queryBuilder().sql(sql).rowSet();
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+            .addNullable("Number", TypeProtos.MinorType.INT)
+            .addNullable("Letter", TypeProtos.MinorType.VARCHAR)
+            .addNullable("Address", TypeProtos.MinorType.INT)
+            .addNullable("Date", TypeProtos.MinorType.DATE)
+            .addNullable("Time",TypeProtos.MinorType.TIME)
+            .addNullable("DateTime",TypeProtos.MinorType.TIMESTAMP)
+            .buildSchema();
+
+
+    RowSet expected = new RowSetBuilder(client.allocator(), expectedSchema)
+            .addRow(1234, "test", 567, LocalDate.parse("2021-02-10"), LocalTime.parse("10:30:27"), Instant.parse("2021-02-10T15:30:27.00Z"))
+            .addRow(5678, "TEST", 890, LocalDate.parse("2021-07-27"), LocalTime.parse("12:40:15"), Instant.parse("2021-07-27T16:40:15.00Z"))
+            .addRow(1111, "abcd", 111, LocalDate.parse("1111-11-11"), LocalTime.parse("11:11:11"), Instant.parse("1111-11-11T16:28:43.11Z"))
+            .addRow(2222, "efgh", 222, LocalDate.parse("2222-01-22"), LocalTime.parse("22:22:22"), Instant.parse("2222-01-23T03:22:22.22Z"))
+            .addRow(3333, "ijkl", 333, LocalDate.parse("3333-02-01"), LocalTime.parse("01:33:33"), Instant.parse("3333-02-01T06:33:33.33Z"))
+            .addRow(4444, "mnop", 444, LocalDate.parse("4444-03-02"), LocalTime.parse("02:44:44"), Instant.parse("4444-03-02T07:44:44.44Z"))
+            .addRow(5555, "qrst", 555, LocalDate.parse("5555-04-03"), LocalTime.parse("03:55:55"), Instant.parse("5555-04-03T07:55:55.55Z"))
+            .addRow(6666, "uvwx", 666, LocalDate.parse("6666-05-04"), LocalTime.parse("04:01:01"), Instant.parse("6666-05-04T08:01:01.01Z"))
+            .addRow(7777, "yzzz", 777, LocalDate.parse("7777-06-05"), LocalTime.parse("05:11:11"), Instant.parse("7777-06-05T09:11:11.11Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .build();
+
+    System.out.println(expected);
+    assertEquals(25, results.rowCount());
+
+    //System.out.println(results.batchSchema());
+    System.out.println(results);
+
+
+    new RowSetComparison(expected).verifyAndClearAll(results);
+    System.out.println("Test complete.");

Review comment:
       These last two lines are not necessary.

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator>{
+
+  private static final Logger logger = LoggerFactory.getLogger(FixedwidthBatchReader.class);
+
+  private FileSplit split;
+
+  private final int maxRecords;
+
+  private final FixedwidthFormatConfig config;
+
+  private CustomErrorContext errorContext;
+
+  private InputStream fsStream;
+
+  private ResultSetLoader loader;
+
+  private BufferedReader reader;
+
+  public FixedwidthBatchReader(FixedwidthFormatConfig config, int maxRecords) {
+    this.config = config;
+    this.maxRecords = maxRecords;
+  }
+
+  @Override
+  public boolean open(FileSchemaNegotiator negotiator) {
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      negotiator.tableSchema(buildSchema(),true);
+      loader = negotiator.build();
+    } catch (Exception e) {
+      throw UserException
+              .dataReadError(e)
+              .message("Failed to open input file: {}", split.getPath().toString())
+              .addContext(errorContext)
+              .addContext(e.getMessage())
+              .build(logger);
+    }
+
+    reader = new BufferedReader(new InputStreamReader(fsStream, Charsets.UTF_8));
+
+    return true;
+
+  }
+
+  @Override
+  public boolean next() { // Use loader to read data from file to turn into Drill rows
+
+    String line;
+
+    try {
+      line = reader.readLine();
+      RowSetLoader writer = loader.writer();
+
+      while (!writer.isFull() && line != null) {
+
+        writer.start();
+        parseLine(line, writer);
+        writer.save();
+
+        line = reader.readLine();
+      }
+    } catch (Exception e) {
+      throw UserException
+              .dataReadError(e)
+              .message("Failed to read input file: {}", split.getPath().toString())
+              .addContext(errorContext)
+              .addContext(e.getMessage())
+              .build(logger);
+    }
+    return (line != null);
+  }
+
+  @Override
+  public void close() {
+    try {
+      fsStream.close();
+      loader.close();
+    } catch (Exception e) {
+      throw UserException
+              .dataReadError(e)
+              .message("Failed to close input file: {}", split.getPath().toString())
+              .addContext(errorContext)
+              .addContext(e.getMessage())
+              .build(logger);
+    }
+  }
+
+  private TupleMetadata buildSchema(){
+    SchemaBuilder builder = new SchemaBuilder();
+
+    for (FixedwidthFieldConfig field : config.getFields()){
+      builder.addNullable(field.getFieldName(),field.getDataType());
+    }
+
+      return builder.buildSchema();
+  }
+
+  private void parseLine(String line, RowSetLoader writer) {

Review comment:
       See comment above but I'd recommend making this method return a boolean value. 

##########
File path: contrib/format-fixedwidth/src/test/java/org/apache/drill/exec/store/fixedwidth/TestFixedwidthRecordReader.java
##########
@@ -0,0 +1,126 @@
+package org.apache.drill.exec.store.fixedwidth;
+
+import org.apache.drill.categories.RowSetTests;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.rowSet.RowSet;
+import org.apache.drill.exec.physical.rowSet.RowSetBuilder;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.drill.test.ClusterFixture;
+import org.apache.drill.test.ClusterTest;
+import org.apache.drill.test.QueryBuilder;
+import org.apache.drill.test.rowSet.RowSetComparison;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.nio.file.Paths;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+
+import static org.junit.Assert.assertEquals;
+
+@Category(RowSetTests.class)
+public class TestFixedwidthRecordReader extends ClusterTest {
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    ClusterTest.startCluster(ClusterFixture.builder(dirTestWatcher));
+
+    FixedwidthFormatConfig formatConfig = new FixedwidthFormatConfig(Lists.newArrayList("fwf"),
+            Lists.newArrayList(
+            new FixedwidthFieldConfig(TypeProtos.MinorType.INT, "Number", "", 1, 4),
+            new FixedwidthFieldConfig(TypeProtos.MinorType.VARCHAR, "Letter", "", 6, 4),
+            new FixedwidthFieldConfig(TypeProtos.MinorType.INT,"Address","",11,3),
+            new FixedwidthFieldConfig(TypeProtos.MinorType.DATE,"Date","MM-dd-yyyy",15,10),
+            new FixedwidthFieldConfig(TypeProtos.MinorType.TIME,"Time","HH:mm:ss",26,8),
+            new FixedwidthFieldConfig(TypeProtos.MinorType.TIMESTAMP,"DateTime","MM-dd-yyyy'T'HH:mm:ss.SSX",35,23)
+    ));
+    cluster.defineFormat("cp", "fwf", formatConfig);
+
+    // Needed for compressed file unit test
+    dirTestWatcher.copyResourceToRoot(Paths.get("fwf/"));
+  }
+
+  @Test
+  public void testExplicitQuery() throws Exception {
+    String sql = "SELECT ID, Urban, Urban_value FROM dfs.`spss/testdata.sav` WHERE d16=4";
+
+    QueryBuilder q = client.queryBuilder().sql(sql);
+    RowSet results = q.rowSet();
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+            .addNullable("ID", TypeProtos.MinorType.FLOAT8)
+            .addNullable("Urban", TypeProtos.MinorType.FLOAT8)
+            .addNullable("Urban_value", TypeProtos.MinorType.VARCHAR)
+            .buildSchema();
+
+
+    RowSet expected = new RowSetBuilder(client.allocator(), expectedSchema)
+            .addRow(47.0, 1.0, "Urban").addRow(53.0, 1.0, "Urban")
+            .addRow(66.0, 1.0, "Urban")
+            .build();
+
+    assertEquals(3, results.rowCount());
+
+    new RowSetComparison(expected).verifyAndClearAll(results);
+  }
+
+  @Test
+  public void testBatchReader() throws Exception {
+    String sql = "SELECT * FROM cp.`fwf/test.fwf`";
+    RowSet results = client.queryBuilder().sql(sql).rowSet();
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+            .addNullable("Number", TypeProtos.MinorType.INT)
+            .addNullable("Letter", TypeProtos.MinorType.VARCHAR)
+            .addNullable("Address", TypeProtos.MinorType.INT)
+            .addNullable("Date", TypeProtos.MinorType.DATE)
+            .addNullable("Time",TypeProtos.MinorType.TIME)
+            .addNullable("DateTime",TypeProtos.MinorType.TIMESTAMP)
+            .buildSchema();
+
+
+    RowSet expected = new RowSetBuilder(client.allocator(), expectedSchema)
+            .addRow(1234, "test", 567, LocalDate.parse("2021-02-10"), LocalTime.parse("10:30:27"), Instant.parse("2021-02-10T15:30:27.00Z"))
+            .addRow(5678, "TEST", 890, LocalDate.parse("2021-07-27"), LocalTime.parse("12:40:15"), Instant.parse("2021-07-27T16:40:15.00Z"))
+            .addRow(1111, "abcd", 111, LocalDate.parse("1111-11-11"), LocalTime.parse("11:11:11"), Instant.parse("1111-11-11T16:28:43.11Z"))
+            .addRow(2222, "efgh", 222, LocalDate.parse("2222-01-22"), LocalTime.parse("22:22:22"), Instant.parse("2222-01-23T03:22:22.22Z"))
+            .addRow(3333, "ijkl", 333, LocalDate.parse("3333-02-01"), LocalTime.parse("01:33:33"), Instant.parse("3333-02-01T06:33:33.33Z"))
+            .addRow(4444, "mnop", 444, LocalDate.parse("4444-03-02"), LocalTime.parse("02:44:44"), Instant.parse("4444-03-02T07:44:44.44Z"))
+            .addRow(5555, "qrst", 555, LocalDate.parse("5555-04-03"), LocalTime.parse("03:55:55"), Instant.parse("5555-04-03T07:55:55.55Z"))
+            .addRow(6666, "uvwx", 666, LocalDate.parse("6666-05-04"), LocalTime.parse("04:01:01"), Instant.parse("6666-05-04T08:01:01.01Z"))
+            .addRow(7777, "yzzz", 777, LocalDate.parse("7777-06-05"), LocalTime.parse("05:11:11"), Instant.parse("7777-06-05T09:11:11.11Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .build();
+
+    System.out.println(expected);
+    assertEquals(25, results.rowCount());
+
+    //System.out.println(results.batchSchema());
+    System.out.println(results);
+
+
+    new RowSetComparison(expected).verifyAndClearAll(results);
+    System.out.println("Test complete.");

Review comment:
       Please add tests for:
   
   * Serialization/Deserialization
   * Compressed file
   * Various invalid schemata.  For instance, what happens if you don't have fields defined in the config and try to query the data?

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthFieldConfig.java
##########
@@ -0,0 +1,71 @@
+package org.apache.drill.exec.store.fixedwidth;

Review comment:
       Please add Apache license to all files. 

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator>{
+
+  private static final Logger logger = LoggerFactory.getLogger(FixedwidthBatchReader.class);
+
+  private FileSplit split;
+
+  private final int maxRecords;
+
+  private final FixedwidthFormatConfig config;
+
+  private CustomErrorContext errorContext;
+
+  private InputStream fsStream;
+
+  private ResultSetLoader loader;
+
+  private BufferedReader reader;
+
+  public FixedwidthBatchReader(FixedwidthFormatConfig config, int maxRecords) {
+    this.config = config;
+    this.maxRecords = maxRecords;
+  }
+
+  @Override
+  public boolean open(FileSchemaNegotiator negotiator) {
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      negotiator.tableSchema(buildSchema(),true);
+      loader = negotiator.build();
+    } catch (Exception e) {
+      throw UserException
+              .dataReadError(e)
+              .message("Failed to open input file: {}", split.getPath().toString())
+              .addContext(errorContext)
+              .addContext(e.getMessage())
+              .build(logger);
+    }
+
+    reader = new BufferedReader(new InputStreamReader(fsStream, Charsets.UTF_8));
+
+    return true;
+
+  }
+
+  @Override
+  public boolean next() { // Use loader to read data from file to turn into Drill rows
+

Review comment:
       
   There is a small issue with the `next()` function as written.  You define `maxRecords` but don't do anything with it.  The `maxRecords` is the `LIMIT` which gets pushed down from the query.   The idea being that if a user does a `SELECT ... LIMIT 10` your reader should stop reading as soon as that limit is reached.  Which means that the `next` function should return `false` when the limit has been reached.   The good news is that your `writer` object actually has a method called `limitReached(<maxRecords>)` which will track this for you.
   
   Another thing you might consider doing which would clean up the code a bit would be to make the `parseLine` method return `true` if there is more data to read, `false` if not.  Also move the writer start and end to that method, then you could have a `next` method that looks like this:
   
   ```java
   @Override
     public boolean next() {
       recordCount = 0;
       while (!writer.isFull()) {
         if (!parseLine(writer)) {
           return false;
         }
       }
       return true;
     }
   ```   
   

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthFieldConfig.java
##########
@@ -0,0 +1,71 @@
+package org.apache.drill.exec.store.fixedwidth;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.drill.common.types.TypeProtos;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName("fixedwidthReaderFieldDescription")
+@JsonInclude(JsonInclude.Include.NON_DEFAULT)

Review comment:
       For all serializable classes in Drill, I would recommend overriding `equals`, `hashcode`, and `toString()` methods. 

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator>{
+
+  private static final Logger logger = LoggerFactory.getLogger(FixedwidthBatchReader.class);
+
+  private FileSplit split;
+
+  private final int maxRecords;
+
+  private final FixedwidthFormatConfig config;
+
+  private CustomErrorContext errorContext;
+
+  private InputStream fsStream;
+
+  private ResultSetLoader loader;
+
+  private BufferedReader reader;
+
+  public FixedwidthBatchReader(FixedwidthFormatConfig config, int maxRecords) {
+    this.config = config;
+    this.maxRecords = maxRecords;
+  }
+
+  @Override
+  public boolean open(FileSchemaNegotiator negotiator) {
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      negotiator.tableSchema(buildSchema(),true);
+      loader = negotiator.build();
+    } catch (Exception e) {
+      throw UserException
+              .dataReadError(e)
+              .message("Failed to open input file: {}", split.getPath().toString())
+              .addContext(errorContext)
+              .addContext(e.getMessage())
+              .build(logger);
+    }
+
+    reader = new BufferedReader(new InputStreamReader(fsStream, Charsets.UTF_8));
+
+    return true;
+
+  }
+
+  @Override
+  public boolean next() { // Use loader to read data from file to turn into Drill rows
+
+    String line;
+
+    try {
+      line = reader.readLine();
+      RowSetLoader writer = loader.writer();
+
+      while (!writer.isFull() && line != null) {
+
+        writer.start();
+        parseLine(line, writer);
+        writer.save();
+
+        line = reader.readLine();
+      }
+    } catch (Exception e) {
+      throw UserException
+              .dataReadError(e)
+              .message("Failed to read input file: {}", split.getPath().toString())
+              .addContext(errorContext)
+              .addContext(e.getMessage())
+              .build(logger);
+    }
+    return (line != null);
+  }
+
+  @Override
+  public void close() {
+    try {
+      fsStream.close();
+      loader.close();
+    } catch (Exception e) {
+      throw UserException
+              .dataReadError(e)
+              .message("Failed to close input file: {}", split.getPath().toString())
+              .addContext(errorContext)
+              .addContext(e.getMessage())
+              .build(logger);
+    }
+  }
+
+  private TupleMetadata buildSchema(){
+    SchemaBuilder builder = new SchemaBuilder();
+
+    for (FixedwidthFieldConfig field : config.getFields()){
+      builder.addNullable(field.getFieldName(),field.getDataType());
+    }
+
+      return builder.buildSchema();
+  }
+
+  private void parseLine(String line, RowSetLoader writer) {
+    int i = 0;
+    TypeProtos.MinorType dataType;
+    String dateTimeFormat;
+    String value;
+
+    for (FixedwidthFieldConfig field : config.getFields()) {
+      value = line.substring(field.getStartIndex() - 1, field.getStartIndex() + field.getFieldWidth() - 1);
+
+      dataType = field.getDataType();
+      dateTimeFormat = field.getDateTimeFormat();
+      DateTimeFormatter formatter = DateTimeFormatter.ofPattern(dateTimeFormat, Locale.ENGLISH);
+
+      switch (dataType) {
+        case INT:
+          writer.scalar(i).setInt(Integer.parseInt(value));
+          break;
+        case VARCHAR:
+          writer.scalar(i).setString(value);
+          break;
+        case DATE:
+          LocalDate date = LocalDate.parse(value, formatter);
+          writer.scalar(i).setDate(date);
+          break;
+        case TIME:
+          LocalTime time = LocalTime.parse(value, formatter);
+          writer.scalar(i).setTime(time);
+          break;
+        case TIMESTAMP:
+          LocalDateTime ldt = LocalDateTime.parse(value,formatter);
+          ZoneId z = ZoneId.of( "America/Toronto" );
+          ZonedDateTime zdt = ldt.atZone( z );
+          Instant timeStamp = zdt.toInstant();
+          writer.scalar(i).setTimestamp(timeStamp);
+          break;
+        default:
+          throw new RuntimeException("Unknown data type specified in fixed width. Found data type " + dataType);

Review comment:
       Please convert this to a `UserException`. 

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator>{
+
+  private static final Logger logger = LoggerFactory.getLogger(FixedwidthBatchReader.class);
+
+  private FileSplit split;
+
+  private final int maxRecords;
+
+  private final FixedwidthFormatConfig config;
+
+  private CustomErrorContext errorContext;
+
+  private InputStream fsStream;
+
+  private ResultSetLoader loader;
+
+  private BufferedReader reader;
+
+  public FixedwidthBatchReader(FixedwidthFormatConfig config, int maxRecords) {
+    this.config = config;
+    this.maxRecords = maxRecords;
+  }
+
+  @Override
+  public boolean open(FileSchemaNegotiator negotiator) {
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      negotiator.tableSchema(buildSchema(),true);
+      loader = negotiator.build();
+    } catch (Exception e) {
+      throw UserException
+              .dataReadError(e)
+              .message("Failed to open input file: {}", split.getPath().toString())
+              .addContext(errorContext)
+              .addContext(e.getMessage())
+              .build(logger);
+    }
+
+    reader = new BufferedReader(new InputStreamReader(fsStream, Charsets.UTF_8));
+
+    return true;
+
+  }
+
+  @Override
+  public boolean next() { // Use loader to read data from file to turn into Drill rows
+
+    String line;
+
+    try {
+      line = reader.readLine();
+      RowSetLoader writer = loader.writer();
+
+      while (!writer.isFull() && line != null) {
+
+        writer.start();
+        parseLine(line, writer);
+        writer.save();
+
+        line = reader.readLine();
+      }
+    } catch (Exception e) {

Review comment:
       I'd suggest tightening this a bit.  Instead of capturing a generic `Exception`, perhaps use specific exceptions with different error messages.  Alternatively just put the try/catch around the line(s) in which data is being read.  

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthFormatPlugin.java
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileReaderFactory;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileScanBuilder;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.options.OptionManager;
+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 FixedwidthFormatPlugin extends EasyFormatPlugin<FixedwidthFormatConfig> {
+
+  protected static final String DEFAULT_NAME = "fixedwidth";
+
+  private static class FixedwidthReaderFactory extends FileReaderFactory {
+
+    private final FixedwidthFormatConfig config;
+    private final int maxRecords;
+
+    public FixedwidthReaderFactory(FixedwidthFormatConfig config, int maxRecords) {
+      this.config = config;
+      this.maxRecords = maxRecords;
+    }
+
+    @Override
+    public ManagedReader<? extends FileSchemaNegotiator> newReader() {
+      return new FixedwidthBatchReader(config, maxRecords);
+    }
+  }
+
+  public FixedwidthFormatPlugin(String name,
+                                DrillbitContext context,
+                                Configuration fsConf,
+                                StoragePluginConfig storageConfig,
+                                FixedwidthFormatConfig formatConfig) {
+    super(name, easyConfig(fsConf, formatConfig), context, storageConfig, formatConfig);
+  } //final?
+
+  private static EasyFormatConfig easyConfig(Configuration fsConf, FixedwidthFormatConfig pluginConfig) {
+    return EasyFormatConfig.builder()
+            .readable(true)
+            .writable(false)
+            .blockSplittable(false)

Review comment:
       I think this actually might be blocksplittable.  

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthFieldConfig.java
##########
@@ -0,0 +1,71 @@
+package org.apache.drill.exec.store.fixedwidth;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.drill.common.types.TypeProtos;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName("fixedwidthReaderFieldDescription")
+@JsonInclude(JsonInclude.Include.NON_DEFAULT)
+public class FixedwidthFieldConfig {
+
+  private final TypeProtos.MinorType dataType;
+  private final String fieldName;
+  private final String dateTimeFormat;
+  private final int startIndex;
+  private final int fieldWidth;
+
+  public FixedwidthFieldConfig(@JsonProperty("dataType") TypeProtos.MinorType dataType,
+                               @JsonProperty("fieldName") String fieldName,
+                               @JsonProperty("dateTimeFormat") String dateTimeFormat,
+                               @JsonProperty("startIndex") int startIndex,
+                               @JsonProperty("fieldWidth") int fieldWidth) {
+    this.dataType = dataType;
+    this.fieldName = fieldName;
+    this.dateTimeFormat = dateTimeFormat;
+    this.startIndex = startIndex;
+    this.fieldWidth = fieldWidth;
+  }
+
+  public TypeProtos.MinorType getDataType(){
+    return dataType;
+  }
+
+//  public void setDataType(TypeProtos.MinorType dataType){

Review comment:
       Please remove struts here and elsewhere.

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator>{
+
+  private static final Logger logger = LoggerFactory.getLogger(FixedwidthBatchReader.class);
+
+  private FileSplit split;
+
+  private final int maxRecords;
+
+  private final FixedwidthFormatConfig config;
+
+  private CustomErrorContext errorContext;
+
+  private InputStream fsStream;
+
+  private ResultSetLoader loader;
+
+  private BufferedReader reader;
+
+  public FixedwidthBatchReader(FixedwidthFormatConfig config, int maxRecords) {
+    this.config = config;
+    this.maxRecords = maxRecords;
+  }
+
+  @Override
+  public boolean open(FileSchemaNegotiator negotiator) {
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      negotiator.tableSchema(buildSchema(),true);
+      loader = negotiator.build();
+    } catch (Exception e) {
+      throw UserException
+              .dataReadError(e)
+              .message("Failed to open input file: {}", split.getPath().toString())
+              .addContext(errorContext)
+              .addContext(e.getMessage())
+              .build(logger);
+    }
+
+    reader = new BufferedReader(new InputStreamReader(fsStream, Charsets.UTF_8));
+

Review comment:
       Nit:  Please remove extra whitespace, here and elsewhere.

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator>{
+
+  private static final Logger logger = LoggerFactory.getLogger(FixedwidthBatchReader.class);
+
+  private FileSplit split;
+
+  private final int maxRecords;
+
+  private final FixedwidthFormatConfig config;
+
+  private CustomErrorContext errorContext;
+
+  private InputStream fsStream;
+
+  private ResultSetLoader loader;
+
+  private BufferedReader reader;
+
+  public FixedwidthBatchReader(FixedwidthFormatConfig config, int maxRecords) {
+    this.config = config;
+    this.maxRecords = maxRecords;
+  }
+
+  @Override
+  public boolean open(FileSchemaNegotiator negotiator) {
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      negotiator.tableSchema(buildSchema(),true);
+      loader = negotiator.build();
+    } catch (Exception e) {
+      throw UserException
+              .dataReadError(e)
+              .message("Failed to open input file: {}", split.getPath().toString())
+              .addContext(errorContext)
+              .addContext(e.getMessage())
+              .build(logger);
+    }
+
+    reader = new BufferedReader(new InputStreamReader(fsStream, Charsets.UTF_8));
+
+    return true;
+
+  }
+
+  @Override
+  public boolean next() { // Use loader to read data from file to turn into Drill rows
+
+    String line;
+
+    try {
+      line = reader.readLine();
+      RowSetLoader writer = loader.writer();
+
+      while (!writer.isFull() && line != null) {
+
+        writer.start();
+        parseLine(line, writer);
+        writer.save();
+
+        line = reader.readLine();
+      }
+    } catch (Exception e) {
+      throw UserException
+              .dataReadError(e)
+              .message("Failed to read input file: {}", split.getPath().toString())
+              .addContext(errorContext)
+              .addContext(e.getMessage())
+              .build(logger);
+    }
+    return (line != null);
+  }
+
+  @Override
+  public void close() {
+    try {
+      fsStream.close();

Review comment:
       Consider using `Autoclosables` here.  That will do all the error handling for you. 
   
   ```java
    @Override
     public void close() {
       if (fsStream != null) {
         AutoCloseables.closeSilently(fsStream);
         fsStream = null;
       }
     }
   ```




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

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

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



[GitHub] [drill] cgivre commented on a change in pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
cgivre commented on a change in pull request #2282:
URL: https://github.com/apache/drill/pull/2282#discussion_r740503925



##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator> {

Review comment:
       @paul-rogers Is there an example somewhere of how to use the rowset loader to set up the schema automatically?  Is this as simple as checking to see whether the schema is provided and if so, use that?




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

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

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



[GitHub] [drill] lgtm-com[bot] commented on pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #2282:
URL: https://github.com/apache/drill/pull/2282#issuecomment-946116674


   This pull request **introduces 2 alerts** when merging 2d17f1baa1595206b02eff9ddbf0723dbbb5e575 into 0c9451e6720e5028e1187067cc6d1957ff998bef - [view on LGTM.com](https://lgtm.com/projects/g/apache/drill/rev/pr-c9ae7f3eacc38ffc9dddd0071aa52ce59a7c752e)
   
   **new alerts:**
   
   * 2 for Unused format argument


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

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

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



[GitHub] [drill] lgtm-com[bot] commented on pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #2282:
URL: https://github.com/apache/drill/pull/2282#issuecomment-948816808


   This pull request **introduces 2 alerts** when merging a91be4c633f8fbbecd64200b5ead07c5244e1a2b into f4ea90ce3e70065c5db364c5f06452c079c151ac - [view on LGTM.com](https://lgtm.com/projects/g/apache/drill/rev/pr-70cf24e0bb175b8cd007daeb533edd346458c7b5)
   
   **new alerts:**
   
   * 2 for Unused format argument


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

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

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



[GitHub] [drill] lgtm-com[bot] commented on pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #2282:
URL: https://github.com/apache/drill/pull/2282#issuecomment-973244742


   This pull request **introduces 2 alerts** when merging 428a512ec35309c90b254c71735f88b97768f7ca into 14d96d1b6a847f3c07a453f6641993da21a4167c - [view on LGTM.com](https://lgtm.com/projects/g/apache/drill/rev/pr-0dea4b746255df65147d73ec7e5b0297db4bb759)
   
   **new alerts:**
   
   * 2 for Unused format argument


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

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

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



[GitHub] [drill] MFoss19 commented on a change in pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
MFoss19 commented on a change in pull request #2282:
URL: https://github.com/apache/drill/pull/2282#discussion_r759499866



##########
File path: contrib/format-fixedwidth/src/test/java/org/apache/drill/exec/store/fixedwidth/TestFixedwidthRecordReader.java
##########
@@ -0,0 +1,221 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.categories.RowSetTests;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.rowSet.RowSet;
+import org.apache.drill.exec.physical.rowSet.RowSetBuilder;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.drill.test.ClusterFixture;
+import org.apache.drill.test.ClusterTest;
+import org.apache.drill.test.QueryBuilder;
+import org.apache.drill.test.rowSet.RowSetComparison;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.nio.file.Paths;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+
+import static org.apache.drill.test.QueryTestUtil.generateCompressedFile;
+import static org.junit.Assert.assertEquals;
+
+@Category(RowSetTests.class)
+public class TestFixedwidthRecordReader extends ClusterTest {
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    ClusterTest.startCluster(ClusterFixture.builder(dirTestWatcher));
+
+    FixedwidthFormatConfig formatConfig = new FixedwidthFormatConfig(Lists.newArrayList("fwf"),
+      Lists.newArrayList(
+        new FixedwidthFieldConfig("Number", 1, 5, TypeProtos.MinorType.VARDECIMAL),
+        new FixedwidthFieldConfig("Address",12, 3,TypeProtos.MinorType.INT,  ""),

Review comment:
       Fixed spacing and removed unnecessary dateTimeFormats




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

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

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



[GitHub] [drill] MFoss19 commented on a change in pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
MFoss19 commented on a change in pull request #2282:
URL: https://github.com/apache/drill/pull/2282#discussion_r684484684



##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator>{
+
+  private static final Logger logger = LoggerFactory.getLogger(FixedwidthBatchReader.class);
+
+  private FileSplit split;
+
+  private final int maxRecords;
+
+  private final FixedwidthFormatConfig config;
+
+  private CustomErrorContext errorContext;
+
+  private InputStream fsStream;
+
+  private ResultSetLoader loader;
+
+  private BufferedReader reader;
+
+  public FixedwidthBatchReader(FixedwidthFormatConfig config, int maxRecords) {
+    this.config = config;
+    this.maxRecords = maxRecords;
+  }
+
+  @Override
+  public boolean open(FileSchemaNegotiator negotiator) {
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      negotiator.tableSchema(buildSchema(),true);
+      loader = negotiator.build();
+    } catch (Exception e) {
+      throw UserException
+              .dataReadError(e)
+              .message("Failed to open input file: {}", split.getPath().toString())
+              .addContext(errorContext)
+              .addContext(e.getMessage())
+              .build(logger);
+    }
+
+    reader = new BufferedReader(new InputStreamReader(fsStream, Charsets.UTF_8));
+
+    return true;
+
+  }
+
+  @Override
+  public boolean next() { // Use loader to read data from file to turn into Drill rows
+
+    String line;
+
+    try {
+      line = reader.readLine();
+      RowSetLoader writer = loader.writer();
+
+      while (!writer.isFull() && line != null) {
+
+        writer.start();
+        parseLine(line, writer);
+        writer.save();
+
+        line = reader.readLine();
+      }
+    } catch (Exception e) {

Review comment:
       Modified to capture IOException. At first, had done this deliberately since parseLine would throw Runtime Exception, but captured this to rethrow as IOException for clarity.




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

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

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



[GitHub] [drill] MFoss19 commented on a change in pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
MFoss19 commented on a change in pull request #2282:
URL: https://github.com/apache/drill/pull/2282#discussion_r731177777



##########
File path: contrib/format-fixedwidth/src/test/java/org/apache/drill/exec/store/fixedwidth/TestFixedwidthRecordReader.java
##########
@@ -0,0 +1,126 @@
+package org.apache.drill.exec.store.fixedwidth;
+
+import org.apache.drill.categories.RowSetTests;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.rowSet.RowSet;
+import org.apache.drill.exec.physical.rowSet.RowSetBuilder;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.drill.test.ClusterFixture;
+import org.apache.drill.test.ClusterTest;
+import org.apache.drill.test.QueryBuilder;
+import org.apache.drill.test.rowSet.RowSetComparison;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.nio.file.Paths;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+
+import static org.junit.Assert.assertEquals;
+
+@Category(RowSetTests.class)
+public class TestFixedwidthRecordReader extends ClusterTest {
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    ClusterTest.startCluster(ClusterFixture.builder(dirTestWatcher));
+
+    FixedwidthFormatConfig formatConfig = new FixedwidthFormatConfig(Lists.newArrayList("fwf"),
+            Lists.newArrayList(
+            new FixedwidthFieldConfig(TypeProtos.MinorType.INT, "Number", "", 1, 4),
+            new FixedwidthFieldConfig(TypeProtos.MinorType.VARCHAR, "Letter", "", 6, 4),
+            new FixedwidthFieldConfig(TypeProtos.MinorType.INT,"Address","",11,3),
+            new FixedwidthFieldConfig(TypeProtos.MinorType.DATE,"Date","MM-dd-yyyy",15,10),
+            new FixedwidthFieldConfig(TypeProtos.MinorType.TIME,"Time","HH:mm:ss",26,8),
+            new FixedwidthFieldConfig(TypeProtos.MinorType.TIMESTAMP,"DateTime","MM-dd-yyyy'T'HH:mm:ss.SSX",35,23)
+    ));
+    cluster.defineFormat("cp", "fwf", formatConfig);
+
+    // Needed for compressed file unit test
+    dirTestWatcher.copyResourceToRoot(Paths.get("fwf/"));
+  }
+
+  @Test
+  public void testExplicitQuery() throws Exception {
+    String sql = "SELECT ID, Urban, Urban_value FROM dfs.`spss/testdata.sav` WHERE d16=4";
+
+    QueryBuilder q = client.queryBuilder().sql(sql);
+    RowSet results = q.rowSet();
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+            .addNullable("ID", TypeProtos.MinorType.FLOAT8)
+            .addNullable("Urban", TypeProtos.MinorType.FLOAT8)
+            .addNullable("Urban_value", TypeProtos.MinorType.VARCHAR)
+            .buildSchema();
+
+
+    RowSet expected = new RowSetBuilder(client.allocator(), expectedSchema)
+            .addRow(47.0, 1.0, "Urban").addRow(53.0, 1.0, "Urban")
+            .addRow(66.0, 1.0, "Urban")
+            .build();
+
+    assertEquals(3, results.rowCount());
+
+    new RowSetComparison(expected).verifyAndClearAll(results);
+  }
+
+  @Test
+  public void testBatchReader() throws Exception {
+    String sql = "SELECT * FROM cp.`fwf/test.fwf`";
+    RowSet results = client.queryBuilder().sql(sql).rowSet();
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+            .addNullable("Number", TypeProtos.MinorType.INT)
+            .addNullable("Letter", TypeProtos.MinorType.VARCHAR)
+            .addNullable("Address", TypeProtos.MinorType.INT)
+            .addNullable("Date", TypeProtos.MinorType.DATE)
+            .addNullable("Time",TypeProtos.MinorType.TIME)
+            .addNullable("DateTime",TypeProtos.MinorType.TIMESTAMP)
+            .buildSchema();
+
+
+    RowSet expected = new RowSetBuilder(client.allocator(), expectedSchema)
+            .addRow(1234, "test", 567, LocalDate.parse("2021-02-10"), LocalTime.parse("10:30:27"), Instant.parse("2021-02-10T15:30:27.00Z"))
+            .addRow(5678, "TEST", 890, LocalDate.parse("2021-07-27"), LocalTime.parse("12:40:15"), Instant.parse("2021-07-27T16:40:15.00Z"))
+            .addRow(1111, "abcd", 111, LocalDate.parse("1111-11-11"), LocalTime.parse("11:11:11"), Instant.parse("1111-11-11T16:28:43.11Z"))
+            .addRow(2222, "efgh", 222, LocalDate.parse("2222-01-22"), LocalTime.parse("22:22:22"), Instant.parse("2222-01-23T03:22:22.22Z"))
+            .addRow(3333, "ijkl", 333, LocalDate.parse("3333-02-01"), LocalTime.parse("01:33:33"), Instant.parse("3333-02-01T06:33:33.33Z"))
+            .addRow(4444, "mnop", 444, LocalDate.parse("4444-03-02"), LocalTime.parse("02:44:44"), Instant.parse("4444-03-02T07:44:44.44Z"))
+            .addRow(5555, "qrst", 555, LocalDate.parse("5555-04-03"), LocalTime.parse("03:55:55"), Instant.parse("5555-04-03T07:55:55.55Z"))
+            .addRow(6666, "uvwx", 666, LocalDate.parse("6666-05-04"), LocalTime.parse("04:01:01"), Instant.parse("6666-05-04T08:01:01.01Z"))
+            .addRow(7777, "yzzz", 777, LocalDate.parse("7777-06-05"), LocalTime.parse("05:11:11"), Instant.parse("7777-06-05T09:11:11.11Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .build();
+
+    System.out.println(expected);
+    assertEquals(25, results.rowCount());
+
+    //System.out.println(results.batchSchema());

Review comment:
       Done

##########
File path: contrib/format-fixedwidth/src/test/java/org/apache/drill/exec/store/fixedwidth/TestFixedwidthRecordReader.java
##########
@@ -0,0 +1,126 @@
+package org.apache.drill.exec.store.fixedwidth;
+
+import org.apache.drill.categories.RowSetTests;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.rowSet.RowSet;
+import org.apache.drill.exec.physical.rowSet.RowSetBuilder;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.drill.test.ClusterFixture;
+import org.apache.drill.test.ClusterTest;
+import org.apache.drill.test.QueryBuilder;
+import org.apache.drill.test.rowSet.RowSetComparison;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.nio.file.Paths;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+
+import static org.junit.Assert.assertEquals;
+
+@Category(RowSetTests.class)
+public class TestFixedwidthRecordReader extends ClusterTest {
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    ClusterTest.startCluster(ClusterFixture.builder(dirTestWatcher));
+
+    FixedwidthFormatConfig formatConfig = new FixedwidthFormatConfig(Lists.newArrayList("fwf"),
+            Lists.newArrayList(
+            new FixedwidthFieldConfig(TypeProtos.MinorType.INT, "Number", "", 1, 4),
+            new FixedwidthFieldConfig(TypeProtos.MinorType.VARCHAR, "Letter", "", 6, 4),
+            new FixedwidthFieldConfig(TypeProtos.MinorType.INT,"Address","",11,3),
+            new FixedwidthFieldConfig(TypeProtos.MinorType.DATE,"Date","MM-dd-yyyy",15,10),
+            new FixedwidthFieldConfig(TypeProtos.MinorType.TIME,"Time","HH:mm:ss",26,8),
+            new FixedwidthFieldConfig(TypeProtos.MinorType.TIMESTAMP,"DateTime","MM-dd-yyyy'T'HH:mm:ss.SSX",35,23)
+    ));
+    cluster.defineFormat("cp", "fwf", formatConfig);
+
+    // Needed for compressed file unit test
+    dirTestWatcher.copyResourceToRoot(Paths.get("fwf/"));
+  }
+
+  @Test
+  public void testExplicitQuery() throws Exception {
+    String sql = "SELECT ID, Urban, Urban_value FROM dfs.`spss/testdata.sav` WHERE d16=4";
+
+    QueryBuilder q = client.queryBuilder().sql(sql);
+    RowSet results = q.rowSet();
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+            .addNullable("ID", TypeProtos.MinorType.FLOAT8)
+            .addNullable("Urban", TypeProtos.MinorType.FLOAT8)
+            .addNullable("Urban_value", TypeProtos.MinorType.VARCHAR)
+            .buildSchema();
+
+
+    RowSet expected = new RowSetBuilder(client.allocator(), expectedSchema)
+            .addRow(47.0, 1.0, "Urban").addRow(53.0, 1.0, "Urban")
+            .addRow(66.0, 1.0, "Urban")
+            .build();
+
+    assertEquals(3, results.rowCount());
+
+    new RowSetComparison(expected).verifyAndClearAll(results);
+  }
+
+  @Test
+  public void testBatchReader() throws Exception {
+    String sql = "SELECT * FROM cp.`fwf/test.fwf`";
+    RowSet results = client.queryBuilder().sql(sql).rowSet();
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+            .addNullable("Number", TypeProtos.MinorType.INT)
+            .addNullable("Letter", TypeProtos.MinorType.VARCHAR)
+            .addNullable("Address", TypeProtos.MinorType.INT)
+            .addNullable("Date", TypeProtos.MinorType.DATE)
+            .addNullable("Time",TypeProtos.MinorType.TIME)
+            .addNullable("DateTime",TypeProtos.MinorType.TIMESTAMP)
+            .buildSchema();
+
+
+    RowSet expected = new RowSetBuilder(client.allocator(), expectedSchema)
+            .addRow(1234, "test", 567, LocalDate.parse("2021-02-10"), LocalTime.parse("10:30:27"), Instant.parse("2021-02-10T15:30:27.00Z"))
+            .addRow(5678, "TEST", 890, LocalDate.parse("2021-07-27"), LocalTime.parse("12:40:15"), Instant.parse("2021-07-27T16:40:15.00Z"))
+            .addRow(1111, "abcd", 111, LocalDate.parse("1111-11-11"), LocalTime.parse("11:11:11"), Instant.parse("1111-11-11T16:28:43.11Z"))
+            .addRow(2222, "efgh", 222, LocalDate.parse("2222-01-22"), LocalTime.parse("22:22:22"), Instant.parse("2222-01-23T03:22:22.22Z"))
+            .addRow(3333, "ijkl", 333, LocalDate.parse("3333-02-01"), LocalTime.parse("01:33:33"), Instant.parse("3333-02-01T06:33:33.33Z"))
+            .addRow(4444, "mnop", 444, LocalDate.parse("4444-03-02"), LocalTime.parse("02:44:44"), Instant.parse("4444-03-02T07:44:44.44Z"))
+            .addRow(5555, "qrst", 555, LocalDate.parse("5555-04-03"), LocalTime.parse("03:55:55"), Instant.parse("5555-04-03T07:55:55.55Z"))
+            .addRow(6666, "uvwx", 666, LocalDate.parse("6666-05-04"), LocalTime.parse("04:01:01"), Instant.parse("6666-05-04T08:01:01.01Z"))
+            .addRow(7777, "yzzz", 777, LocalDate.parse("7777-06-05"), LocalTime.parse("05:11:11"), Instant.parse("7777-06-05T09:11:11.11Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .addRow(8888, "aabb", 888, LocalDate.parse("8888-07-06"), LocalTime.parse("06:22:22"), Instant.parse("8888-07-07T10:22:22.22Z"))
+            .build();
+
+    System.out.println(expected);
+    assertEquals(25, results.rowCount());
+
+    //System.out.println(results.batchSchema());
+    System.out.println(results);
+
+
+    new RowSetComparison(expected).verifyAndClearAll(results);
+    System.out.println("Test complete.");

Review comment:
       Done




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

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

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



[GitHub] [drill] lgtm-com[bot] commented on pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #2282:
URL: https://github.com/apache/drill/pull/2282#issuecomment-961938801






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

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

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



[GitHub] [drill] lgtm-com[bot] commented on pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #2282:
URL: https://github.com/apache/drill/pull/2282#issuecomment-978491665


   This pull request **introduces 2 alerts** when merging 56d8f6e79d5e33470c120d3071495f631d6f3d1b into 38d0c1d9586b7adc8adfb89f574085cdc26595fe - [view on LGTM.com](https://lgtm.com/projects/g/apache/drill/rev/pr-676a753c0777e9b3f1c3535f1f15f620049cbaf4)
   
   **new alerts:**
   
   * 2 for Unused format argument


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

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

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



[GitHub] [drill] MFoss19 commented on a change in pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
MFoss19 commented on a change in pull request #2282:
URL: https://github.com/apache/drill/pull/2282#discussion_r681090696



##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator>{
+
+  private static final Logger logger = LoggerFactory.getLogger(FixedwidthBatchReader.class);
+
+  private FileSplit split;
+
+  private final int maxRecords;
+
+  private final FixedwidthFormatConfig config;
+
+  private CustomErrorContext errorContext;
+
+  private InputStream fsStream;
+
+  private ResultSetLoader loader;
+
+  private BufferedReader reader;
+
+  public FixedwidthBatchReader(FixedwidthFormatConfig config, int maxRecords) {
+    this.config = config;
+    this.maxRecords = maxRecords;
+  }
+
+  @Override
+  public boolean open(FileSchemaNegotiator negotiator) {
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      negotiator.tableSchema(buildSchema(),true);
+      loader = negotiator.build();
+    } catch (Exception e) {
+      throw UserException
+              .dataReadError(e)
+              .message("Failed to open input file: {}", split.getPath().toString())
+              .addContext(errorContext)
+              .addContext(e.getMessage())
+              .build(logger);
+    }
+
+    reader = new BufferedReader(new InputStreamReader(fsStream, Charsets.UTF_8));
+

Review comment:
       Done




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

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

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



[GitHub] [drill] dzamo edited a comment on pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
dzamo edited a comment on pull request #2282:
URL: https://github.com/apache/drill/pull/2282#issuecomment-962939201


   @MFoss19 @estherbuchwalter following some [recent chat](https://github.com/apache/drill/pull/2359#issuecomment-962673076) with @paul-rogers and my last comment here, how about a reduced format config such as the following?  The goal is to get to something terse and consistent with what we do for other text formats.
   
   ```json
   "fixedwidth": {
     "type": "fixedwidth",
     "extensions": [
       "fwf"
     ],
     "extractHeader": true,
     "trimStrings": true,
     "columnOffsets": [1, 11, 21, 31],
     "columnWidths": [10, 10, 10, 10]
   }
   ```
   
   Column names and types can already come from a provided schema or aliasing after calls to `CAST()`.  Incidentally, the settings above can be overriden per query using a provided schema too.
   
   There's also a part of that wonders whether we could have justified adding our fixed width functionality to the existing delimited text format reader.


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

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

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



[GitHub] [drill] lgtm-com[bot] commented on pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #2282:
URL: https://github.com/apache/drill/pull/2282#issuecomment-961938801


   This pull request **introduces 2 alerts** when merging 9d66f9114ab27429a8366d1d71b15f638574dc45 into 52838ef26e5e3e6b4461c2c656ffada0c64c9e88 - [view on LGTM.com](https://lgtm.com/projects/g/apache/drill/rev/pr-d2e0dd9a19bfba83486250ec80ec4737d67ff38e)
   
   **new alerts:**
   
   * 2 for Unused format argument


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

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

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



[GitHub] [drill] MFoss19 commented on a change in pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
MFoss19 commented on a change in pull request #2282:
URL: https://github.com/apache/drill/pull/2282#discussion_r686046450



##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator>{
+
+  private static final Logger logger = LoggerFactory.getLogger(FixedwidthBatchReader.class);
+
+  private FileSplit split;
+
+  private final int maxRecords;
+
+  private final FixedwidthFormatConfig config;
+
+  private CustomErrorContext errorContext;
+
+  private InputStream fsStream;
+
+  private ResultSetLoader loader;
+
+  private BufferedReader reader;
+
+  public FixedwidthBatchReader(FixedwidthFormatConfig config, int maxRecords) {
+    this.config = config;
+    this.maxRecords = maxRecords;
+  }
+
+  @Override
+  public boolean open(FileSchemaNegotiator negotiator) {
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      negotiator.tableSchema(buildSchema(),true);
+      loader = negotiator.build();
+    } catch (Exception e) {
+      throw UserException
+              .dataReadError(e)
+              .message("Failed to open input file: {}", split.getPath().toString())
+              .addContext(errorContext)
+              .addContext(e.getMessage())
+              .build(logger);
+    }
+
+    reader = new BufferedReader(new InputStreamReader(fsStream, Charsets.UTF_8));
+
+    return true;
+
+  }
+
+  @Override
+  public boolean next() { // Use loader to read data from file to turn into Drill rows
+
+    String line;
+
+    try {
+      line = reader.readLine();
+      RowSetLoader writer = loader.writer();
+
+      while (!writer.isFull() && line != null) {
+
+        writer.start();
+        parseLine(line, writer);
+        writer.save();
+
+        line = reader.readLine();
+      }
+    } catch (Exception e) {
+      throw UserException
+              .dataReadError(e)
+              .message("Failed to read input file: {}", split.getPath().toString())
+              .addContext(errorContext)
+              .addContext(e.getMessage())
+              .build(logger);
+    }
+    return (line != null);
+  }
+
+  @Override
+  public void close() {
+    try {
+      fsStream.close();

Review comment:
       Done




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

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

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



[GitHub] [drill] paul-rogers commented on a change in pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
paul-rogers commented on a change in pull request #2282:
URL: https://github.com/apache/drill/pull/2282#discussion_r757130589



##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.math.BigDecimal;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(FixedwidthBatchReader.class);
+  private FileSplit split;
+  private final int maxRecords;
+  private final FixedwidthFormatConfig config;
+  private CustomErrorContext errorContext;
+  private InputStream fsStream;
+  private ResultSetLoader loader;
+  private RowSetLoader writer;
+  private BufferedReader reader;
+  private int lineNum;
+
+  public FixedwidthBatchReader(FixedwidthFormatConfig config, int maxRecords) {
+    this.config = config;
+    this.maxRecords = maxRecords;
+  }
+
+  @Override
+  public boolean open(FileSchemaNegotiator negotiator) {
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    lineNum = 0;
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      negotiator.tableSchema(buildSchema(), true);
+      loader = negotiator.build();
+    } catch (Exception e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to open input file: {}", split.getPath().toString())
+        .addContext(errorContext)
+        .addContext(e.getMessage())
+        .build(logger);
+    }
+    reader = new BufferedReader(new InputStreamReader(fsStream, Charsets.UTF_8));
+    return true;
+  }
+
+  @Override
+  public boolean next() { // Use loader to read data from file to turn into Drill rows
+    String line;
+    RowSetLoader writer = loader.writer();
+
+    try {
+      line = reader.readLine();
+      while (!writer.isFull() && line != null) {
+        writer.start();
+        parseLine(line, writer);
+        writer.save();
+        line = reader.readLine();

Review comment:
       You can lose a line here: read the next line, but the writer is full, so we discard it. As @cgivre suggests:
   
   ```java
   while (!writer.isFull()) {
     String line = reader.readLine();
     if (line == null) {
       break;
     }
     writer.start();
     parseLine(line, writer);
     writer.save();
   }
   ```    

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthFormatConfig.java
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.fixedwidth;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.common.PlanStringBuilder;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.logical.FormatPluginConfig;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.regex.Pattern;
+
+@JsonTypeName(FixedwidthFormatPlugin.DEFAULT_NAME)
+@JsonInclude(JsonInclude.Include.NON_DEFAULT)
+public class FixedwidthFormatConfig implements FormatPluginConfig {
+  private static final Logger logger = LoggerFactory.getLogger(FixedwidthFormatConfig.class);
+  private final List<String> extensions;
+  private final List<FixedwidthFieldConfig> fields;
+  private final List<TypeProtos.MinorType> validDataTypes = Arrays.asList(TypeProtos.MinorType.INT, TypeProtos.MinorType.VARCHAR,
+    TypeProtos.MinorType.DATE, TypeProtos.MinorType.TIME, TypeProtos.MinorType.TIMESTAMP, TypeProtos.MinorType.FLOAT4,
+    TypeProtos.MinorType.FLOAT8, TypeProtos.MinorType.BIGINT, TypeProtos.MinorType.VARDECIMAL);
+
+  @JsonCreator
+  public FixedwidthFormatConfig(@JsonProperty("extensions") List<String> extensions,
+                                @JsonProperty("fields") List<FixedwidthFieldConfig> fields) {
+    this.extensions = extensions == null ? Collections.singletonList("fwf") : ImmutableList.copyOf(extensions);
+    Collections.sort(fields);
+    this.fields = fields;
+
+    validateFieldInput();

Review comment:
       This is just a bit dangerous. We validate on deserialize. This seems like a great idea: we do the deserialize in the UI when the user saves the config. But, we also do it on system start. If the deserialize fails there, Drill won't start and it takes a long time to figure out why.
   
   We don't have a good answer for config validation. I'd suggest adding a `validate()` method that we call:
   
   * When saving a config from the UI
   * When planning a query using a config
   
   But, allow an invalid config to be stored. Otherwise, unless everything is perfect, nothing can be saved. And, JSON, in a text editor, is a horrible way to write complex config such as this one.

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.math.BigDecimal;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(FixedwidthBatchReader.class);
+  private FileSplit split;
+  private final int maxRecords;
+  private final FixedwidthFormatConfig config;
+  private CustomErrorContext errorContext;
+  private InputStream fsStream;
+  private ResultSetLoader loader;
+  private RowSetLoader writer;
+  private BufferedReader reader;
+  private int lineNum;
+
+  public FixedwidthBatchReader(FixedwidthFormatConfig config, int maxRecords) {
+    this.config = config;
+    this.maxRecords = maxRecords;
+  }
+
+  @Override
+  public boolean open(FileSchemaNegotiator negotiator) {
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    lineNum = 0;
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      negotiator.tableSchema(buildSchema(), true);
+      loader = negotiator.build();
+    } catch (Exception e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to open input file: {}", split.getPath().toString())
+        .addContext(errorContext)
+        .addContext(e.getMessage())
+        .build(logger);
+    }
+    reader = new BufferedReader(new InputStreamReader(fsStream, Charsets.UTF_8));
+    return true;
+  }
+
+  @Override
+  public boolean next() { // Use loader to read data from file to turn into Drill rows
+    String line;
+    RowSetLoader writer = loader.writer();
+
+    try {
+      line = reader.readLine();
+      while (!writer.isFull() && line != null) {
+        writer.start();
+        parseLine(line, writer);
+        writer.save();
+        line = reader.readLine();
+        lineNum++;
+      }
+    } catch (IOException e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to read input file: {}", split.getPath().toString())
+        .addContext(errorContext)
+        .addContext(e.getMessage())
+        .addContext("Line Number", lineNum)
+        .build(logger);
+    }
+    return writer.limitReached(maxRecords);  // returns false when maxRecords limit has been reached
+  }
+
+  @Override
+  public void close() {
+    if (fsStream != null){
+      AutoCloseables.closeSilently(fsStream);

Review comment:
       The reason it an be out of the `if` statement is that the method itself handles null values. Otherwise, the code would be fine as it is if it called `close()` directly.

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.math.BigDecimal;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(FixedwidthBatchReader.class);
+  private FileSplit split;
+  private final int maxRecords;
+  private final FixedwidthFormatConfig config;
+  private CustomErrorContext errorContext;
+  private InputStream fsStream;
+  private ResultSetLoader loader;
+  private RowSetLoader writer;
+  private BufferedReader reader;
+  private int lineNum;
+
+  public FixedwidthBatchReader(FixedwidthFormatConfig config, int maxRecords) {
+    this.config = config;
+    this.maxRecords = maxRecords;
+  }
+
+  @Override
+  public boolean open(FileSchemaNegotiator negotiator) {
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    lineNum = 0;
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());

Review comment:
       Fixed-width files seem perfect for HDFS splits.
   
   So there is a trick here for old-school HDFS systems (if anyone still runs them.) A large file will be split across HDFS nodes, often at 256MB boundaries. A reader that supports "splits" has to handle the fact that node 1 will read the first 256 MB, node 2 the next 256 MB, and so on. Standard HDFS stuff.
   
   This means the reader has to accept the offset and scan for the first record separator after that start point. If the fixed-width records are newline-terminated, that means finding the next newline.
   
   Also, the reader has to stop after it finds the record terminator after its assigned split. (On HDFS, that means part of the last record will be read from a remote HDFS node.)
   
   Further, it means that line numbers, as counted here, are relative: they are from the start of the current split (since node 2, reading the second split, doesn't know the number of records in the first split.)
   
   The CSV reader handles this busy-work, but it a way that is a bit hard to follow as an example, sadly.

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthFieldConfig.java
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.fixedwidth;
+
+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 org.apache.drill.common.PlanStringBuilder;
+import org.apache.drill.common.types.TypeProtos;
+
+import java.util.Objects;
+
+
+@JsonTypeName("fixedwidthReaderFieldDescription")
+@JsonInclude(JsonInclude.Include.NON_DEFAULT)
+public class FixedwidthFieldConfig implements Comparable<FixedwidthFieldConfig> {
+
+  private final String name;
+  private final int index;
+  private int width;
+  private TypeProtos.MinorType type;
+  private final String dateTimeFormat;
+
+  public FixedwidthFieldConfig(@JsonProperty("name") String name,
+                               @JsonProperty("index") int index,
+                               @JsonProperty("width") int width,
+                               @JsonProperty("type") TypeProtos.MinorType type) {
+    this(name, index, width, type, null);
+  }
+
+  @JsonCreator
+  public FixedwidthFieldConfig(@JsonProperty("name") String name,
+                               @JsonProperty("index") int index,
+                               @JsonProperty("width") int width,

Review comment:
       This makes the user do the math. Given indexes, we can compute the width. Given widths, we can compute the index. The one missing piece might the the width of the field separator (if any).

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthFieldConfig.java
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.fixedwidth;
+
+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 org.apache.drill.common.PlanStringBuilder;
+import org.apache.drill.common.types.TypeProtos;
+
+import java.util.Objects;
+
+
+@JsonTypeName("fixedwidthReaderFieldDescription")
+@JsonInclude(JsonInclude.Include.NON_DEFAULT)
+public class FixedwidthFieldConfig implements Comparable<FixedwidthFieldConfig> {

Review comment:
       Just a high-level design note: we now have multiple plugin configs that ask the user to use ad-hoc formats for specifying a per-file schema. This is not how storage plugins were meant to be used, but it is all we have.
   
   Over time, we need a better solution. The provided-schema is a start: it provides a single, well-defines syntax for schema. But, it is a bit limited in handling per-format specific, such as width. (There are extension properties for this kind of information, but that's a bit fiddly.)
   
   We need a way to provide the schema of a file simply as a file the user creates that sits along side the data file. Something like a `schema.json` file if all files in a directory have the same schema, or a `foo.json` if `foo.txt` has a distinct schema.
   
   Support for such a file should be added to the V2 mechanism. (Just use the new format in place of the provided schema.)

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthFormatPlugin.java
##########
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.drill.exec.store.fixedwidth;
+
+import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileReaderFactory;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileScanBuilder;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.options.OptionManager;
+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 FixedwidthFormatPlugin extends EasyFormatPlugin<FixedwidthFormatConfig> {
+
+  protected static final String DEFAULT_NAME = "fixedwidth";
+
+  private static class FixedwidthReaderFactory extends FileReaderFactory {
+
+    private final FixedwidthFormatConfig config;
+    private final int maxRecords;
+
+    public FixedwidthReaderFactory(FixedwidthFormatConfig config, int maxRecords) {
+      this.config = config;
+      this.maxRecords = maxRecords;
+    }
+
+    @Override
+    public ManagedReader<? extends FileSchemaNegotiator> newReader() {
+      return new FixedwidthBatchReader(config, maxRecords);
+    }
+  }
+
+  public FixedwidthFormatPlugin(String name,
+                                DrillbitContext context,
+                                Configuration fsConf,
+                                StoragePluginConfig storageConfig,
+                                FixedwidthFormatConfig formatConfig) {
+    super(name, easyConfig(fsConf, formatConfig), context, storageConfig, formatConfig);
+  }
+
+  private static EasyFormatConfig easyConfig(Configuration fsConf, FixedwidthFormatConfig pluginConfig) {
+    return EasyFormatConfig.builder()
+      .readable(true)
+      .writable(false)
+      .blockSplittable(false) // Change to true
+      .compressible(true)
+      .supportsProjectPushdown(true)
+      .extensions(pluginConfig.getExtensions())
+      .fsConf(fsConf)
+      .defaultName(DEFAULT_NAME)
+      .useEnhancedScan(true)
+      .supportsLimitPushdown(true)
+      .build();
+  }
+
+  @Override
+  public ManagedReader<? extends FileSchemaNegotiator> newBatchReader(
+    EasySubScan scan, OptionManager options) {
+    return new FixedwidthBatchReader(getConfig(), scan.getMaxRecords());
+  }
+
+  @Override
+  protected FileScanBuilder frameworkBuilder(OptionManager options, EasySubScan scan) {
+    FileScanBuilder builder = new FileScanBuilder();
+    builder.setReaderFactory(new FixedwidthReaderFactory(getConfig(), scan.getMaxRecords()));
+    initScanBuilder(builder, scan);
+    builder.nullType(Types.optional(TypeProtos.MinorType.VARCHAR));
+    return builder;
+  }
+}

Review comment:
       Nit: missing newline.

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthFormatConfig.java
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.fixedwidth;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.common.PlanStringBuilder;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.logical.FormatPluginConfig;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.regex.Pattern;
+
+@JsonTypeName(FixedwidthFormatPlugin.DEFAULT_NAME)
+@JsonInclude(JsonInclude.Include.NON_DEFAULT)
+public class FixedwidthFormatConfig implements FormatPluginConfig {
+  private static final Logger logger = LoggerFactory.getLogger(FixedwidthFormatConfig.class);
+  private final List<String> extensions;
+  private final List<FixedwidthFieldConfig> fields;
+  private final List<TypeProtos.MinorType> validDataTypes = Arrays.asList(TypeProtos.MinorType.INT, TypeProtos.MinorType.VARCHAR,
+    TypeProtos.MinorType.DATE, TypeProtos.MinorType.TIME, TypeProtos.MinorType.TIMESTAMP, TypeProtos.MinorType.FLOAT4,
+    TypeProtos.MinorType.FLOAT8, TypeProtos.MinorType.BIGINT, TypeProtos.MinorType.VARDECIMAL);
+
+  @JsonCreator
+  public FixedwidthFormatConfig(@JsonProperty("extensions") List<String> extensions,
+                                @JsonProperty("fields") List<FixedwidthFieldConfig> fields) {
+    this.extensions = extensions == null ? Collections.singletonList("fwf") : ImmutableList.copyOf(extensions);
+    Collections.sort(fields);
+    this.fields = fields;
+
+    validateFieldInput();
+  }
+
+  @JsonInclude(JsonInclude.Include.NON_DEFAULT)
+  public List<String> getExtensions() {
+    return extensions;
+  }
+
+  public List<FixedwidthFieldConfig> getFields() {
+    return fields;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(extensions, fields);
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null || getClass() != obj.getClass()) {
+      return false;
+    }
+    FixedwidthFormatConfig other = (FixedwidthFormatConfig) obj;
+    return Objects.equals(extensions, other.extensions)
+            && Objects.equals(fields, other.fields);
+  }
+
+  @Override
+  public String toString() {
+    return new PlanStringBuilder(this)
+            .field("extensions", extensions)
+            .field("fields", fields)
+            .toString();
+  }
+
+
+  @JsonIgnore
+  public boolean hasFields() {
+    return fields != null && ! fields.isEmpty();
+  }
+
+  @JsonIgnore
+  public List<String> getFieldNames() {
+    List<String> result = new ArrayList<>();
+    if (! hasFields()) {
+      return result;
+    }
+
+    for (FixedwidthFieldConfig field : fields) {
+      result.add(field.getName());
+    }
+    return result;
+  }
+
+  @JsonIgnore
+  public List<Integer> getFieldIndices() {
+    List<Integer> result = new ArrayList<>();
+    if (! hasFields()) {
+      return result;
+    }
+
+    for (FixedwidthFieldConfig field : fields) {
+      result.add(field.getIndex());
+    }
+    return result;
+  }
+
+  @JsonIgnore
+  public List<Integer> getFieldWidths() {
+    List<Integer> result = new ArrayList<>();
+    if (! hasFields()) {
+      return result;
+    }
+
+    for (FixedwidthFieldConfig field : fields) {
+      result.add(field.getWidth());
+    }
+    return result;
+  }
+
+  @JsonIgnore
+  public List<TypeProtos.MinorType> getFieldTypes() {
+    List<TypeProtos.MinorType> result = new ArrayList<>();
+    if (! hasFields()) {
+      return result;
+    }
+
+    for (FixedwidthFieldConfig field : fields) {
+      result.add(field.getType());
+    }
+    return result;
+  }
+
+  @JsonIgnore
+  public void setFieldTypes(int i) {
+    for (FixedwidthFieldConfig field : fields) {
+      if (field.getIndex() == i) {
+        field.setType();
+      }
+    }
+  }
+
+  @JsonIgnore

Review comment:
       No need for this tag: Jackson doesn't know what to do with this method anyway.

##########
File path: contrib/format-fixedwidth/src/test/java/org/apache/drill/exec/store/fixedwidth/TestFixedwidthRecordReader.java
##########
@@ -0,0 +1,221 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.categories.RowSetTests;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.rowSet.RowSet;
+import org.apache.drill.exec.physical.rowSet.RowSetBuilder;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.drill.test.ClusterFixture;
+import org.apache.drill.test.ClusterTest;
+import org.apache.drill.test.QueryBuilder;
+import org.apache.drill.test.rowSet.RowSetComparison;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.nio.file.Paths;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+
+import static org.apache.drill.test.QueryTestUtil.generateCompressedFile;
+import static org.junit.Assert.assertEquals;
+
+@Category(RowSetTests.class)
+public class TestFixedwidthRecordReader extends ClusterTest {
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    ClusterTest.startCluster(ClusterFixture.builder(dirTestWatcher));
+
+    FixedwidthFormatConfig formatConfig = new FixedwidthFormatConfig(Lists.newArrayList("fwf"),
+      Lists.newArrayList(
+        new FixedwidthFieldConfig("Number", 1, 5, TypeProtos.MinorType.VARDECIMAL),
+        new FixedwidthFieldConfig("Address",12, 3,TypeProtos.MinorType.INT,  ""),

Review comment:
       Nit: spacing after comma. Remove unwanted format as per previous line.

##########
File path: contrib/format-fixedwidth/src/test/resources/fwf/test_blankrow.fwf
##########
@@ -0,0 +1,26 @@
+12.34 test 567 02-10-2021 10:30:27 02-10-2021T10:30:27.00Z
+56.78 TEST 890 07-27-2021 12:40:15 07-27-2021T12:40:15.00Z
+11.11 abcd 111 11-11-1111 11:11:11 11-11-1111T11:11:11.11Z
+22.22 efgh 222 01-22-2222 22:22:22 01-22-2222T22:22:22.22Z
+33.33 ijkl 333 02-01-3333 01:33:33 02-01-3333T01:33:33.33Z
+44.44 mnop 444 03-02-4444 02:44:44 03-02-4444T02:44:44.44Z
+55.55 qrst 555 04-03-5555 03:55:55 04-03-5555T03:55:55.55Z
+66.66 uvwx 666 05-04-6666 04:01:01 05-04-6666T04:01:01.01Z
+77.77 yzzz 777 06-05-7777 05:11:11 06-05-7777T05:11:11.11Z
+88.88 aabb 888 07-06-8888 06:22:22 07-07-8888T06:22:22.22Z
+88.88 aabb 888 07-06-8888 06:22:22 07-07-8888T06:22:22.22Z
+

Review comment:
       Is the blank line intentional? Kind of hard to document data files. You could allow comments (lines that start with `#`, say, but there is always someone who has valid data that starts with `#`.) Another idea would be to add a "explanation.txt" file that explains the purpose of each of the data files.

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(FixedwidthBatchReader.class);
+  private FileSplit split;
+  private final int maxRecords;
+  private final FixedwidthFormatConfig config;
+  private CustomErrorContext errorContext;
+  private InputStream fsStream;
+  private ResultSetLoader loader;
+  private RowSetLoader writer;
+  private BufferedReader reader;
+  private int lineNum;
+
+  public FixedwidthBatchReader(FixedwidthFormatConfig config, int maxRecords) {
+    this.config = config;
+    this.maxRecords = maxRecords;
+  }
+
+  @Override
+  public boolean open(FileSchemaNegotiator negotiator) {
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    lineNum = 0;
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      negotiator.tableSchema(buildSchema(), true);
+      loader = negotiator.build();
+    } catch (Exception e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to open input file: {}", split.getPath().toString())
+        .addContext(errorContext)
+        .addContext(e.getMessage())
+        .build(logger);
+    }
+    reader = new BufferedReader(new InputStreamReader(fsStream, Charsets.UTF_8));
+    return true;
+  }
+
+  @Override
+  public boolean next() { // Use loader to read data from file to turn into Drill rows
+    String line;
+    RowSetLoader writer = loader.writer();
+
+    try {
+      line = reader.readLine();
+      while (!writer.isFull() && line != null) {
+        writer.start();
+        parseLine(line, writer);
+        writer.save();
+        line = reader.readLine();
+        lineNum++;
+      }
+    } catch (IOException e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to read input file: {}", split.getPath().toString())
+        .addContext(errorContext)
+        .addContext(e.getMessage())
+        .addContext("Line Number", lineNum)
+        .build(logger);
+    }
+    return writer.limitReached(maxRecords);  // returns false when maxRecords limit has been reached
+  }
+
+  @Override
+  public void close() {
+    try {
+      fsStream.close();
+      loader.close();
+    } catch (Exception e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to close input file: {}", split.getPath().toString())
+        .addContext(errorContext)
+        .addContext(e.getMessage())
+        .build(logger);
+    }
+  }
+
+  private TupleMetadata buildSchema() {
+    SchemaBuilder builder = new SchemaBuilder();
+    for (FixedwidthFieldConfig field : config.getFields()) {
+      builder.addNullable(field.getFieldName(), field.getDataType());
+    }
+    return builder.buildSchema();
+  }
+
+
+  private boolean parseLine(String line, RowSetLoader writer) throws IOException {
+    int i = 0;
+    TypeProtos.MinorType dataType;
+    String dateTimeFormat;
+    String value;
+    for (FixedwidthFieldConfig field : config.getFields()) {
+      value = line.substring(field.getStartIndex() - 1, field.getStartIndex() + field.getFieldWidth() - 1);
+      dataType = field.getDataType();
+      dateTimeFormat = field.getDateTimeFormat();
+      DateTimeFormatter formatter = DateTimeFormatter.ofPattern(dateTimeFormat, Locale.ENGLISH);
+      try {
+        switch (dataType) {

Review comment:
       To be clearer: what we want is to minimize the per-field work. Ideally, we'd set up an array of column converters so the loop looks like:
   
   for (int i = 0; i < config.getFields().size(); i++) {
     FixedwidthFieldConfig field = config.getFields().get(i);
     value = line.substring(field.getIndex() - 1, field.getIndex() + field.getWidth() - 1);
     writer.scalar(i).setString(value)
   }
   ```
   
   However, the above requires the user to specify the config. For every file. On every Drill cluster. Better, if the schema is not given, infer it from the first (few) row(s). Then, set up runtime field objects:
   
   ```java
   for (FieldReader fieldReader : fieldReaders) {
     fieldReader.load(line);
   }
   ```
   
   The field reader:
   
   * Remembers the start and offset
   * Pulls out the string, handling a truncated line
   * Calls a cached column writer
   
   @cgivre mentioned the idea of a column converter. There is a defined set for the common cases. The underlying mechanism sets them up for you. (V2 makes it simpler.) That way, a call to `setString()` directly invokes the thing that converts from string and writes the resulting value: no per-column switch needed.

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthFormatConfig.java
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.fixedwidth;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.common.PlanStringBuilder;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.logical.FormatPluginConfig;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.regex.Pattern;
+
+@JsonTypeName(FixedwidthFormatPlugin.DEFAULT_NAME)
+@JsonInclude(JsonInclude.Include.NON_DEFAULT)
+public class FixedwidthFormatConfig implements FormatPluginConfig {
+  private static final Logger logger = LoggerFactory.getLogger(FixedwidthFormatConfig.class);
+  private final List<String> extensions;
+  private final List<FixedwidthFieldConfig> fields;
+  private final List<TypeProtos.MinorType> validDataTypes = Arrays.asList(TypeProtos.MinorType.INT, TypeProtos.MinorType.VARCHAR,
+    TypeProtos.MinorType.DATE, TypeProtos.MinorType.TIME, TypeProtos.MinorType.TIMESTAMP, TypeProtos.MinorType.FLOAT4,
+    TypeProtos.MinorType.FLOAT8, TypeProtos.MinorType.BIGINT, TypeProtos.MinorType.VARDECIMAL);
+
+  @JsonCreator
+  public FixedwidthFormatConfig(@JsonProperty("extensions") List<String> extensions,
+                                @JsonProperty("fields") List<FixedwidthFieldConfig> fields) {
+    this.extensions = extensions == null ? Collections.singletonList("fwf") : ImmutableList.copyOf(extensions);
+    Collections.sort(fields);
+    this.fields = fields;
+
+    validateFieldInput();
+  }
+
+  @JsonInclude(JsonInclude.Include.NON_DEFAULT)
+  public List<String> getExtensions() {
+    return extensions;
+  }
+
+  public List<FixedwidthFieldConfig> getFields() {
+    return fields;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(extensions, fields);
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null || getClass() != obj.getClass()) {
+      return false;
+    }
+    FixedwidthFormatConfig other = (FixedwidthFormatConfig) obj;
+    return Objects.equals(extensions, other.extensions)
+            && Objects.equals(fields, other.fields);
+  }
+
+  @Override
+  public String toString() {
+    return new PlanStringBuilder(this)
+            .field("extensions", extensions)
+            .field("fields", fields)
+            .toString();
+  }
+
+
+  @JsonIgnore
+  public boolean hasFields() {
+    return fields != null && ! fields.isEmpty();
+  }
+
+  @JsonIgnore
+  public List<String> getFieldNames() {
+    List<String> result = new ArrayList<>();
+    if (! hasFields()) {
+      return result;
+    }
+
+    for (FixedwidthFieldConfig field : fields) {
+      result.add(field.getName());
+    }
+    return result;
+  }
+
+  @JsonIgnore
+  public List<Integer> getFieldIndices() {
+    List<Integer> result = new ArrayList<>();
+    if (! hasFields()) {
+      return result;
+    }
+
+    for (FixedwidthFieldConfig field : fields) {
+      result.add(field.getIndex());
+    }
+    return result;
+  }
+
+  @JsonIgnore
+  public List<Integer> getFieldWidths() {
+    List<Integer> result = new ArrayList<>();
+    if (! hasFields()) {
+      return result;
+    }
+
+    for (FixedwidthFieldConfig field : fields) {
+      result.add(field.getWidth());
+    }
+    return result;
+  }
+
+  @JsonIgnore
+  public List<TypeProtos.MinorType> getFieldTypes() {
+    List<TypeProtos.MinorType> result = new ArrayList<>();
+    if (! hasFields()) {
+      return result;
+    }
+
+    for (FixedwidthFieldConfig field : fields) {
+      result.add(field.getType());
+    }
+    return result;
+  }
+
+  @JsonIgnore
+  public void setFieldTypes(int i) {
+    for (FixedwidthFieldConfig field : fields) {
+      if (field.getIndex() == i) {
+        field.setType();
+      }
+    }
+  }
+
+  @JsonIgnore
+  public void validateFieldInput(){
+    Set<String> uniqueNames = new HashSet<>();
+    List<Integer> fieldIndices = this.getFieldIndices();
+    List<Integer> fieldWidths = this.getFieldWidths();
+    List<String> fieldNames = this.getFieldNames();
+    List<TypeProtos.MinorType> fieldTypes = this.getFieldTypes();
+    int prevIndexAndWidth = -1;
+
+    /* Validate Field Name - Ensure field is not empty, does not exceed maximum length,
+    is valid SQL syntax, and no two fields have the same name
+     */
+    for (String name : this.getFieldNames()){
+      if (name.length() == 0){
+        throw UserException
+          .validationError()
+          .message("Blank field name detected.")
+          .addContext("Plugin", FixedwidthFormatPlugin.DEFAULT_NAME)
+          .build(logger);
+      }
+      if (name.length() > 1024) {
+        throw UserException
+          .validationError()
+          .message("Exceeds maximum length of 1024 characters: " + name.substring(0, 1024))
+          .addContext("Plugin", FixedwidthFormatPlugin.DEFAULT_NAME)
+          .build(logger);
+      }
+      if (!Pattern.matches("[a-zA-Z]\\w*", name)) {
+        throw UserException
+          .validationError()
+          .message("Invalid input: " + name)

Review comment:
       This message will (I hope) be shown to the poor user trying to get a config right in the Drill web UI. Can we be more specific? Such as, "Name is not valid. Only letters allowed."
   
   But, then, why don't we allow numbers or underscores? "Field_3"? If we allowed that, you could use the [Java method](https://docs.oracle.com/javase/8/docs/api/javax/lang/model/SourceVersion.html#isName-java.lang.CharSequence-). Or, extend the pattern.
   
   Also, it is cleaner to use `name = name.strip()` to remove both leading and trailing whitespace so we don't leave whitespace in the name. Otherwise, all users have to know to do their own `strip()` call.

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.math.BigDecimal;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(FixedwidthBatchReader.class);
+  private FileSplit split;
+  private final int maxRecords;
+  private final FixedwidthFormatConfig config;
+  private CustomErrorContext errorContext;
+  private InputStream fsStream;
+  private ResultSetLoader loader;
+  private RowSetLoader writer;
+  private BufferedReader reader;
+  private int lineNum;
+
+  public FixedwidthBatchReader(FixedwidthFormatConfig config, int maxRecords) {
+    this.config = config;
+    this.maxRecords = maxRecords;
+  }
+
+  @Override
+  public boolean open(FileSchemaNegotiator negotiator) {
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    lineNum = 0;
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      negotiator.tableSchema(buildSchema(), true);
+      loader = negotiator.build();
+    } catch (Exception e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to open input file: {}", split.getPath().toString())
+        .addContext(errorContext)
+        .addContext(e.getMessage())

Review comment:
       I like that second line (if you mean the `message` call). Another solution is to change `message()` to `addContext()`. This way, we preserve the message from the actual error, and add context to explain the source of the error. Then, as Charles suggested, we don't need the `addContext(e.getMessage())` bit.

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.math.BigDecimal;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(FixedwidthBatchReader.class);
+  private FileSplit split;
+  private final int maxRecords;
+  private final FixedwidthFormatConfig config;
+  private CustomErrorContext errorContext;
+  private InputStream fsStream;
+  private ResultSetLoader loader;
+  private RowSetLoader writer;
+  private BufferedReader reader;
+  private int lineNum;
+
+  public FixedwidthBatchReader(FixedwidthFormatConfig config, int maxRecords) {
+    this.config = config;
+    this.maxRecords = maxRecords;
+  }
+
+  @Override
+  public boolean open(FileSchemaNegotiator negotiator) {
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    lineNum = 0;
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      negotiator.tableSchema(buildSchema(), true);

Review comment:
       This is done internally in V2. V2 does it that way because it became clear that there is no reason for every reader to have to know to do this same pattern.

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,200 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.math.BigDecimal;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(FixedwidthBatchReader.class);
+  private FileSplit split;
+  private final int maxRecords;
+  private final FixedwidthFormatConfig config;
+  private CustomErrorContext errorContext;
+  private InputStream fsStream;
+  private ResultSetLoader loader;
+  private RowSetLoader writer;
+  private BufferedReader reader;
+  private int lineNum;
+
+  public FixedwidthBatchReader(FixedwidthFormatConfig config, int maxRecords) {
+    this.config = config;
+    this.maxRecords = maxRecords;
+  }
+
+
+  @Override
+  public boolean open(FileSchemaNegotiator negotiator) {
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    lineNum = 0;
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      negotiator.tableSchema(buildSchema(), true);
+      loader = negotiator.build();
+    } catch (Exception e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to open input file: {}", split.getPath().toString())
+        .addContext(errorContext)
+        .addContext(e.getMessage())
+        .build(logger);
+    }
+    reader = new BufferedReader(new InputStreamReader(fsStream, Charsets.UTF_8));
+    return true;
+  }
+
+  @Override
+  public boolean next() { // Use loader to read data from file to turn into Drill rows
+    String line;
+    RowSetLoader writer = loader.writer();
+
+    try {
+      line = reader.readLine();
+      while (!writer.isFull() && line != null) {
+        writer.start();
+        parseLine(line, writer);
+        writer.save();
+        line = reader.readLine();
+        lineNum++;
+      }
+    } catch (IOException e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to read input file: {}", split.getPath().toString())
+        .addContext(errorContext)
+        .addContext(e.getMessage())
+        .addContext("Line Number", lineNum)
+        .build(logger);
+    }
+    return writer.limitReached(maxRecords);  // returns false when maxRecords limit has been reached
+  }
+
+  @Override
+  public void close() {
+    if (fsStream != null){
+      AutoCloseables.closeSilently(fsStream);
+      fsStream = null;
+    }
+  }
+
+  private TupleMetadata buildSchema() {
+    SchemaBuilder builder = new SchemaBuilder();
+    for (FixedwidthFieldConfig field : config.getFields()) {
+      if (field.getType() == TypeProtos.MinorType.VARDECIMAL){
+        builder.addNullable(field.getName(), TypeProtos.MinorType.VARDECIMAL,38,4);
+        //revisit this
+      } else {
+        builder.addNullable(field.getName(), field.getType());
+      }
+    }
+    return builder.buildSchema();
+  }
+
+
+  private boolean parseLine(String line, RowSetLoader writer) throws IOException {
+    int i = 0;
+    TypeProtos.MinorType dataType;
+    String dateTimeFormat;
+    String value;
+    for (FixedwidthFieldConfig field : config.getFields()) {

Review comment:
       What you want to do is to resolve the schema at open time, not when parsing. At open time, you can:
   
   * Get the schema from the plugin, if provided, and use that as the schema.
   * Else, sample the first line to infer the schema.
   
   Since this is a fixed format, we don't want to rediscover the schema on every line: that costs too much. (Think of the case of reading 100M or 1B rows: optimizing the inner loop is critical.)

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.math.BigDecimal;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(FixedwidthBatchReader.class);
+  private FileSplit split;
+  private final int maxRecords;
+  private final FixedwidthFormatConfig config;
+  private CustomErrorContext errorContext;
+  private InputStream fsStream;
+  private ResultSetLoader loader;
+  private RowSetLoader writer;
+  private BufferedReader reader;
+  private int lineNum;
+
+  public FixedwidthBatchReader(FixedwidthFormatConfig config, int maxRecords) {
+    this.config = config;
+    this.maxRecords = maxRecords;
+  }
+
+  @Override
+  public boolean open(FileSchemaNegotiator negotiator) {
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    lineNum = 0;
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      negotiator.tableSchema(buildSchema(), true);
+      loader = negotiator.build();
+    } catch (Exception e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to open input file: {}", split.getPath().toString())
+        .addContext(errorContext)
+        .addContext(e.getMessage())
+        .build(logger);
+    }
+    reader = new BufferedReader(new InputStreamReader(fsStream, Charsets.UTF_8));
+    return true;
+  }
+
+  @Override
+  public boolean next() { // Use loader to read data from file to turn into Drill rows
+    String line;
+    RowSetLoader writer = loader.writer();
+
+    try {
+      line = reader.readLine();
+      while (!writer.isFull() && line != null) {
+        writer.start();
+        parseLine(line, writer);
+        writer.save();
+        line = reader.readLine();
+        lineNum++;
+      }
+    } catch (IOException e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to read input file: {}", split.getPath().toString())
+        .addContext(errorContext)
+        .addContext(e.getMessage())

Review comment:
       See explanation above. Ideally:
   
   ```java
         throw UserException
           .dataReadError(e)
           .addContext("Failed to read input file: {}", split.getPath().toString())
           .addContext(errorContext)
           .addContext("Line Number", lineNum)
           .build(logger);
   ```
   
   Thanks for adding the line number: nice touch.

##########
File path: contrib/format-fixedwidth/src/test/resources/fwf/test_blankrow.fwf
##########
@@ -0,0 +1,26 @@
+12.34 test 567 02-10-2021 10:30:27 02-10-2021T10:30:27.00Z
+56.78 TEST 890 07-27-2021 12:40:15 07-27-2021T12:40:15.00Z
+11.11 abcd 111 11-11-1111 11:11:11 11-11-1111T11:11:11.11Z
+22.22 efgh 222 01-22-2222 22:22:22 01-22-2222T22:22:22.22Z
+33.33 ijkl 333 02-01-3333 01:33:33 02-01-3333T01:33:33.33Z
+44.44 mnop 444 03-02-4444 02:44:44 03-02-4444T02:44:44.44Z
+55.55 qrst 555 04-03-5555 03:55:55 04-03-5555T03:55:55.55Z
+66.66 uvwx 666 05-04-6666 04:01:01 05-04-6666T04:01:01.01Z
+77.77 yzzz 777 06-05-7777 05:11:11 06-05-7777T05:11:11.11Z
+88.88 aabb 888 07-06-8888 06:22:22 07-07-8888T06:22:22.22Z
+88.88 aabb 888 07-06-8888 06:22:22 07-07-8888T06:22:22.22Z
+

Review comment:
       Also, I'd hoped to see all manner of invalid files:
   
   * Truncated line (not just an entirely blank line)
   * String where a number should be.
   * Badly formatted date or time.
   
   Such files will ensure that the error handling works and will raise that perpetual question: if we're a hundred million lines into a fixed-width file, and we hit an error, should we ignore that line and move to the next, or should we fail the query?

##########
File path: contrib/format-fixedwidth/src/test/java/org/apache/drill/exec/store/fixedwidth/TestFixedwidthRecordReader.java
##########
@@ -0,0 +1,221 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.categories.RowSetTests;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.rowSet.RowSet;
+import org.apache.drill.exec.physical.rowSet.RowSetBuilder;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.drill.test.ClusterFixture;
+import org.apache.drill.test.ClusterTest;
+import org.apache.drill.test.QueryBuilder;
+import org.apache.drill.test.rowSet.RowSetComparison;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.nio.file.Paths;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+
+import static org.apache.drill.test.QueryTestUtil.generateCompressedFile;
+import static org.junit.Assert.assertEquals;
+
+@Category(RowSetTests.class)
+public class TestFixedwidthRecordReader extends ClusterTest {
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    ClusterTest.startCluster(ClusterFixture.builder(dirTestWatcher));
+
+    FixedwidthFormatConfig formatConfig = new FixedwidthFormatConfig(Lists.newArrayList("fwf"),
+      Lists.newArrayList(
+        new FixedwidthFieldConfig("Number", 1, 5, TypeProtos.MinorType.VARDECIMAL),
+        new FixedwidthFieldConfig("Address",12, 3,TypeProtos.MinorType.INT,  ""),
+        new FixedwidthFieldConfig("Letter", 7,4, TypeProtos.MinorType.VARCHAR, ""),
+        new FixedwidthFieldConfig("Date",16, 10,TypeProtos.MinorType.DATE,  "MM-dd-yyyy"),
+        new FixedwidthFieldConfig( "Time", 27, 8,TypeProtos.MinorType.TIME,"HH:mm:ss" ),
+        new FixedwidthFieldConfig("DateTime", 36, 23,TypeProtos.MinorType.TIMESTAMP, "MM-dd-yyyy'T'HH:mm:ss.SSX" )
+      ));
+    cluster.defineFormat("dfs", "fwf", formatConfig);
+    cluster.defineFormat("cp", "fwf", formatConfig);
+
+    // Needed for compressed file unit test
+    dirTestWatcher.copyResourceToRoot(Paths.get("fwf/"));
+  }
+
+  @Test
+  public void testStarQuery() throws Exception {
+    String sql = "SELECT * FROM cp.`fwf/test.fwf`";
+    RowSet results = client.queryBuilder().sql(sql).rowSet();
+    RowSet expected = setupTestData();
+    new RowSetComparison(expected).verifyAndClearAll(results);
+  }
+
+  @Test
+  public void testExplicitAllQuery() throws Exception {
+    String sql = "SELECT Number, Letter, Address, `Date`, `Time`, DateTime FROM cp.`fwf/test.fwf`";
+    RowSet results = client.queryBuilder().sql(sql).rowSet();
+    RowSet expected = setupTestData();
+    new RowSetComparison(expected).verifyAndClearAll(results);
+  }
+
+  @Test
+  public void testExplicitQuery() throws Exception {
+    String sql = "SELECT Number, Letter, Address FROM cp.`fwf/test.fwf` WHERE Letter='yzzz'";
+    QueryBuilder q = client.queryBuilder().sql(sql);
+    RowSet results = q.rowSet();
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+      .addNullable("Number", TypeProtos.MinorType.VARDECIMAL,38,4)
+      .addNullable("Letter", TypeProtos.MinorType.VARCHAR)
+      .addNullable("Address", TypeProtos.MinorType.INT)
+      .buildSchema();
+    RowSet expected = new RowSetBuilder(client.allocator(), expectedSchema)
+      .addRow(77.77, "yzzz", 777)
+      .build();
+
+    new RowSetComparison(expected).verifyAndClearAll(results);
+  }
+
+  //Test Serialization/Deserialization
+  @Test
+  public void testSerDe() throws Exception {
+    String sql = "SELECT COUNT(*) FROM cp.`fwf/test.fwf`";
+    String plan = queryBuilder().sql(sql).explainJson();
+    long cnt = queryBuilder().physical(plan).singletonLong();
+    assertEquals(25L, cnt);
+  }
+
+  @Test
+  public void testStarQueryWithCompressedFile() throws Exception {
+    generateCompressedFile("fwf/test.fwf", "zip", "fwf/test.fwf.zip" );
+
+    String sql = "SELECT * FROM dfs.`fwf/test.fwf.zip`";
+    RowSet results = client.queryBuilder().sql(sql).rowSet();
+    RowSet expected = setupTestData();
+    new RowSetComparison(expected).verifyAndClearAll(results);
+  }
+
+  // Test Entering invalid schemata - incorrect limits
+    // Undefined field, what happens
+    // Parse invalid file, make sure correct error
+
+
+  @Test
+  public void testOutOfOrder() throws Exception{
+    String sql = "SELECT Address, DateTime, `Date`, Letter FROM cp.`fwf/test.fwf`";
+    QueryBuilder q = client.queryBuilder().sql(sql);
+    RowSet results = q.rowSet();
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+      .addNullable("Address", TypeProtos.MinorType.INT)
+      .addNullable("DateTime", TypeProtos.MinorType.TIMESTAMP)
+      .addNullable("Date", TypeProtos.MinorType.DATE)
+      .addNullable("Letter", TypeProtos.MinorType.VARCHAR)
+      .buildSchema();
+    RowSet expected = new RowSetBuilder(client.allocator(), expectedSchema)
+      .addRow(567, Instant.parse("2021-02-10T15:30:27.00Z"), LocalDate.parse("2021-02-10"), "test")

Review comment:
       Generally, for something like this, you can use a `LIMIT 2` and only check the first row or two. The code is not going to change column order after the 10th row!

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator> {

Review comment:
       @cgivre, it's been a while since I looked at that stuff. As I recall, the CSV reader was converted, but it is a bit obscure. The unit test also show what can be done, IIRC.
   
   Basically, V2 gets schema from multiple sources:
   
   * Schema info in the plan object (if available in the future)
   * Provided schema (the current interim solution to system-provided schema)
   * Schema implied by the selected columns (for example, a[1] means a has to be an array.)
   * Schema defined by the data source itself (as in Parquet, or in CSV where you can choose the `columns` array)
   * Schema discovered by the traditional "schema on read"
   
   The V2 `SchemaNegotiator` provides the needed methods. V2 will then come up with the final schema. V2 lets you read the entire row (columns a, b, c, say) even if the query wants only column a: V2 silently discards the unwanted columns.

##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthFormatPlugin.java
##########
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.drill.exec.store.fixedwidth;
+
+import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileReaderFactory;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileScanBuilder;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.options.OptionManager;
+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 FixedwidthFormatPlugin extends EasyFormatPlugin<FixedwidthFormatConfig> {
+
+  protected static final String DEFAULT_NAME = "fixedwidth";
+
+  private static class FixedwidthReaderFactory extends FileReaderFactory {
+
+    private final FixedwidthFormatConfig config;
+    private final int maxRecords;
+
+    public FixedwidthReaderFactory(FixedwidthFormatConfig config, int maxRecords) {
+      this.config = config;
+      this.maxRecords = maxRecords;
+    }
+
+    @Override
+    public ManagedReader<? extends FileSchemaNegotiator> newReader() {
+      return new FixedwidthBatchReader(config, maxRecords);
+    }
+  }
+
+  public FixedwidthFormatPlugin(String name,
+                                DrillbitContext context,
+                                Configuration fsConf,
+                                StoragePluginConfig storageConfig,
+                                FixedwidthFormatConfig formatConfig) {
+    super(name, easyConfig(fsConf, formatConfig), context, storageConfig, formatConfig);
+  }
+
+  private static EasyFormatConfig easyConfig(Configuration fsConf, FixedwidthFormatConfig pluginConfig) {
+    return EasyFormatConfig.builder()
+      .readable(true)
+      .writable(false)
+      .blockSplittable(false) // Change to true

Review comment:
       But only if the additional work described above is done.




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

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

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



[GitHub] [drill] MFoss19 commented on a change in pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
MFoss19 commented on a change in pull request #2282:
URL: https://github.com/apache/drill/pull/2282#discussion_r681087216



##########
File path: contrib/format-fixedwidth/pom.xml
##########
@@ -0,0 +1,84 @@
+<?xml version="1.0"?>
+<!--
+
+    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.
+
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">

Review comment:
       Done




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

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

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



[GitHub] [drill] MFoss19 commented on a change in pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
MFoss19 commented on a change in pull request #2282:
URL: https://github.com/apache/drill/pull/2282#discussion_r684486433



##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator>{
+
+  private static final Logger logger = LoggerFactory.getLogger(FixedwidthBatchReader.class);
+
+  private FileSplit split;
+
+  private final int maxRecords;
+
+  private final FixedwidthFormatConfig config;
+
+  private CustomErrorContext errorContext;
+
+  private InputStream fsStream;
+
+  private ResultSetLoader loader;
+
+  private BufferedReader reader;
+
+  public FixedwidthBatchReader(FixedwidthFormatConfig config, int maxRecords) {
+    this.config = config;
+    this.maxRecords = maxRecords;
+  }
+
+  @Override
+  public boolean open(FileSchemaNegotiator negotiator) {
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      negotiator.tableSchema(buildSchema(),true);
+      loader = negotiator.build();
+    } catch (Exception e) {
+      throw UserException
+              .dataReadError(e)
+              .message("Failed to open input file: {}", split.getPath().toString())
+              .addContext(errorContext)
+              .addContext(e.getMessage())
+              .build(logger);
+    }
+
+    reader = new BufferedReader(new InputStreamReader(fsStream, Charsets.UTF_8));
+
+    return true;
+
+  }
+
+  @Override
+  public boolean next() { // Use loader to read data from file to turn into Drill rows
+
+    String line;
+
+    try {
+      line = reader.readLine();
+      RowSetLoader writer = loader.writer();
+
+      while (!writer.isFull() && line != null) {
+
+        writer.start();
+        parseLine(line, writer);
+        writer.save();
+
+        line = reader.readLine();
+      }
+    } catch (Exception e) {
+      throw UserException
+              .dataReadError(e)
+              .message("Failed to read input file: {}", split.getPath().toString())
+              .addContext(errorContext)
+              .addContext(e.getMessage())
+              .build(logger);
+    }
+    return (line != null);
+  }
+
+  @Override
+  public void close() {
+    try {
+      fsStream.close();
+      loader.close();
+    } catch (Exception e) {
+      throw UserException
+              .dataReadError(e)
+              .message("Failed to close input file: {}", split.getPath().toString())
+              .addContext(errorContext)
+              .addContext(e.getMessage())
+              .build(logger);
+    }
+  }
+
+  private TupleMetadata buildSchema(){
+    SchemaBuilder builder = new SchemaBuilder();
+
+    for (FixedwidthFieldConfig field : config.getFields()){
+      builder.addNullable(field.getFieldName(),field.getDataType());
+    }
+
+      return builder.buildSchema();
+  }
+
+  private void parseLine(String line, RowSetLoader writer) {
+    int i = 0;
+    TypeProtos.MinorType dataType;
+    String dateTimeFormat;
+    String value;
+
+    for (FixedwidthFieldConfig field : config.getFields()) {
+      value = line.substring(field.getStartIndex() - 1, field.getStartIndex() + field.getFieldWidth() - 1);
+
+      dataType = field.getDataType();
+      dateTimeFormat = field.getDateTimeFormat();
+      DateTimeFormatter formatter = DateTimeFormatter.ofPattern(dateTimeFormat, Locale.ENGLISH);
+
+      switch (dataType) {
+        case INT:
+          writer.scalar(i).setInt(Integer.parseInt(value));
+          break;
+        case VARCHAR:
+          writer.scalar(i).setString(value);
+          break;
+        case DATE:
+          LocalDate date = LocalDate.parse(value, formatter);
+          writer.scalar(i).setDate(date);
+          break;
+        case TIME:
+          LocalTime time = LocalTime.parse(value, formatter);
+          writer.scalar(i).setTime(time);
+          break;
+        case TIMESTAMP:
+          LocalDateTime ldt = LocalDateTime.parse(value,formatter);
+          ZoneId z = ZoneId.of( "America/Toronto" );
+          ZonedDateTime zdt = ldt.atZone( z );
+          Instant timeStamp = zdt.toInstant();
+          writer.scalar(i).setTimestamp(timeStamp);
+          break;
+        default:
+          throw new RuntimeException("Unknown data type specified in fixed width. Found data type " + dataType);
+

Review comment:
       Done




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

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

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



[GitHub] [drill] lgtm-com[bot] commented on pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #2282:
URL: https://github.com/apache/drill/pull/2282#issuecomment-948040369


   This pull request **introduces 2 alerts** when merging 18380eaff486389417f6349164450ed54c35e342 into f4ea90ce3e70065c5db364c5f06452c079c151ac - [view on LGTM.com](https://lgtm.com/projects/g/apache/drill/rev/pr-e71ca345d89ffe93d6515367433683134fdadd4b)
   
   **new alerts:**
   
   * 2 for Unused format argument


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

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

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



[GitHub] [drill] paul-rogers commented on pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
paul-rogers commented on pull request #2282:
URL: https://github.com/apache/drill/pull/2282#issuecomment-993895683


   @dzamo, thanks for pushing the EVF V2 stuff forward! The `EasyFormatPlugin` should contain "shims" for the original format, for EVF1 and for EVF2. Given that you said you can't find it, and that the CSV reader is still based on V1, I wonder if there is some branch that never got pushed a PR? I'll do some research to determine what's what.


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

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

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



[GitHub] [drill] lgtm-com[bot] commented on pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #2282:
URL: https://github.com/apache/drill/pull/2282#issuecomment-977234939


   This pull request **introduces 2 alerts** when merging 428a2dd2b6823cd33775746a2933a7bf6dcf702b into 17f3654919b8429b12e321fb4ee837b7a52e06f1 - [view on LGTM.com](https://lgtm.com/projects/g/apache/drill/rev/pr-9331a5cc140b147684153d0af6dfdde69dc78826)
   
   **new alerts:**
   
   * 2 for Unused format argument


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

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

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



[GitHub] [drill] lgtm-com[bot] commented on pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #2282:
URL: https://github.com/apache/drill/pull/2282#issuecomment-961938801






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

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

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



[GitHub] [drill] MFoss19 commented on a change in pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
MFoss19 commented on a change in pull request #2282:
URL: https://github.com/apache/drill/pull/2282#discussion_r759498226



##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthFormatPlugin.java
##########
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.drill.exec.store.fixedwidth;
+
+import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileReaderFactory;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileScanBuilder;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.options.OptionManager;
+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 FixedwidthFormatPlugin extends EasyFormatPlugin<FixedwidthFormatConfig> {
+
+  protected static final String DEFAULT_NAME = "fixedwidth";
+
+  private static class FixedwidthReaderFactory extends FileReaderFactory {
+
+    private final FixedwidthFormatConfig config;
+    private final int maxRecords;
+
+    public FixedwidthReaderFactory(FixedwidthFormatConfig config, int maxRecords) {
+      this.config = config;
+      this.maxRecords = maxRecords;
+    }
+
+    @Override
+    public ManagedReader<? extends FileSchemaNegotiator> newReader() {
+      return new FixedwidthBatchReader(config, maxRecords);
+    }
+  }
+
+  public FixedwidthFormatPlugin(String name,
+                                DrillbitContext context,
+                                Configuration fsConf,
+                                StoragePluginConfig storageConfig,
+                                FixedwidthFormatConfig formatConfig) {
+    super(name, easyConfig(fsConf, formatConfig), context, storageConfig, formatConfig);
+  }
+
+  private static EasyFormatConfig easyConfig(Configuration fsConf, FixedwidthFormatConfig pluginConfig) {
+    return EasyFormatConfig.builder()
+      .readable(true)
+      .writable(false)
+      .blockSplittable(false) // Change to true
+      .compressible(true)
+      .supportsProjectPushdown(true)
+      .extensions(pluginConfig.getExtensions())
+      .fsConf(fsConf)
+      .defaultName(DEFAULT_NAME)
+      .useEnhancedScan(true)
+      .supportsLimitPushdown(true)
+      .build();
+  }
+
+  @Override
+  public ManagedReader<? extends FileSchemaNegotiator> newBatchReader(
+    EasySubScan scan, OptionManager options) {
+    return new FixedwidthBatchReader(getConfig(), scan.getMaxRecords());
+  }
+
+  @Override
+  protected FileScanBuilder frameworkBuilder(OptionManager options, EasySubScan scan) {
+    FileScanBuilder builder = new FileScanBuilder();
+    builder.setReaderFactory(new FixedwidthReaderFactory(getConfig(), scan.getMaxRecords()));
+    initScanBuilder(builder, scan);
+    builder.nullType(Types.optional(TypeProtos.MinorType.VARCHAR));
+    return builder;
+  }
+}

Review comment:
       Fixed




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

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

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



[GitHub] [drill] dzamo edited a comment on pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
dzamo edited a comment on pull request #2282:
URL: https://github.com/apache/drill/pull/2282#issuecomment-963159298


   > Let's consider a real world use case: some fixed width log generated by a database. Since the fields may be mashed together, there isn't a delimiter that you can use to divide the fields. You _could_ use however the logRegex reader to do this. That point aside for the moment, the way I imagined someone using this was that different configs could be set up and linked to workspaces such that if a file was in the `mysql_logs` folder, it would use the mysql log config, and if it was in the `postgres` it would use another.
   
   @cgivre  This use case would still work after two `CREATE SCHEMA` statements to set the names and data types, wouldn't it?  The schemas would be applied every subsequent query.
   
   > My opinion here is that the goal should be to get the cleanest data to the user as possible without the user having to rely on CASTs and other complicating factors.
   
   Let's drop the CASTs, those aren't fun.  So we're left with different ways a user can specify column names and types.
   
   1. With a `CREATE SCHEMA` against a directory.
   2. With an inline schema to a table function.
   3. With some plugin-specific format config that works for this plugin but generally not for others.
   
   Any one requires some effort, any one gets you to `select *` returning nice results (disclaimer: is this claim I'm making actually true?) which is super valuable.  So shouldn't we avoid the quirky 3 and commit to 1 and 2 consistently wherever we can?


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

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

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



[GitHub] [drill] MFoss19 commented on a change in pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
MFoss19 commented on a change in pull request #2282:
URL: https://github.com/apache/drill/pull/2282#discussion_r703733950



##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthFieldConfig.java
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.fixedwidth;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.common.types.TypeProtos;
+
+@JsonTypeName("fixedwidthReaderFieldDescription")
+@JsonInclude(JsonInclude.Include.NON_DEFAULT)
+public class FixedwidthFieldConfig {
+
+  private final TypeProtos.MinorType dataType;
+  private final String fieldName;
+  private final String dateTimeFormat;
+  private final int startIndex;
+  private final int fieldWidth;
+
+  public FixedwidthFieldConfig(@JsonProperty("dataType") TypeProtos.MinorType dataType,
+                               @JsonProperty("fieldName") String fieldName,
+                               @JsonProperty("dateTimeFormat") String dateTimeFormat,
+                               @JsonProperty("startIndex") int startIndex,
+                               @JsonProperty("fieldWidth") int fieldWidth) {

Review comment:
       Modified names and order.




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

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

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



[GitHub] [drill] MFoss19 commented on a change in pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
MFoss19 commented on a change in pull request #2282:
URL: https://github.com/apache/drill/pull/2282#discussion_r681092643



##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator>{
+
+  private static final Logger logger = LoggerFactory.getLogger(FixedwidthBatchReader.class);
+
+  private FileSplit split;
+
+  private final int maxRecords;
+
+  private final FixedwidthFormatConfig config;
+
+  private CustomErrorContext errorContext;
+
+  private InputStream fsStream;
+
+  private ResultSetLoader loader;
+
+  private BufferedReader reader;
+
+  public FixedwidthBatchReader(FixedwidthFormatConfig config, int maxRecords) {
+    this.config = config;
+    this.maxRecords = maxRecords;
+  }
+
+  @Override
+  public boolean open(FileSchemaNegotiator negotiator) {
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      negotiator.tableSchema(buildSchema(),true);
+      loader = negotiator.build();
+    } catch (Exception e) {
+      throw UserException
+              .dataReadError(e)
+              .message("Failed to open input file: {}", split.getPath().toString())
+              .addContext(errorContext)
+              .addContext(e.getMessage())
+              .build(logger);
+    }
+
+    reader = new BufferedReader(new InputStreamReader(fsStream, Charsets.UTF_8));
+
+    return true;
+
+  }
+
+  @Override
+  public boolean next() { // Use loader to read data from file to turn into Drill rows
+
+    String line;
+
+    try {
+      line = reader.readLine();
+      RowSetLoader writer = loader.writer();
+
+      while (!writer.isFull() && line != null) {
+
+        writer.start();
+        parseLine(line, writer);
+        writer.save();
+
+        line = reader.readLine();
+      }
+    } catch (Exception e) {
+      throw UserException
+              .dataReadError(e)
+              .message("Failed to read input file: {}", split.getPath().toString())
+              .addContext(errorContext)
+              .addContext(e.getMessage())
+              .build(logger);
+    }
+    return (line != null);
+  }
+
+  @Override
+  public void close() {
+    try {
+      fsStream.close();
+      loader.close();
+    } catch (Exception e) {
+      throw UserException
+              .dataReadError(e)
+              .message("Failed to close input file: {}", split.getPath().toString())
+              .addContext(errorContext)
+              .addContext(e.getMessage())
+              .build(logger);
+    }
+  }
+
+  private TupleMetadata buildSchema(){
+    SchemaBuilder builder = new SchemaBuilder();
+
+    for (FixedwidthFieldConfig field : config.getFields()){
+      builder.addNullable(field.getFieldName(),field.getDataType());
+    }
+
+      return builder.buildSchema();
+  }
+
+  private void parseLine(String line, RowSetLoader writer) {
+    int i = 0;
+    TypeProtos.MinorType dataType;
+    String dateTimeFormat;
+    String value;
+
+    for (FixedwidthFieldConfig field : config.getFields()) {
+      value = line.substring(field.getStartIndex() - 1, field.getStartIndex() + field.getFieldWidth() - 1);
+
+      dataType = field.getDataType();
+      dateTimeFormat = field.getDateTimeFormat();
+      DateTimeFormatter formatter = DateTimeFormatter.ofPattern(dateTimeFormat, Locale.ENGLISH);
+
+      switch (dataType) {
+        case INT:

Review comment:
       Yes, will add LONG and DOUBLE. Also looking to add FLOAT and DECIMAL.




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

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

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



[GitHub] [drill] paul-rogers edited a comment on pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
paul-rogers edited a comment on pull request #2282:
URL: https://github.com/apache/drill/pull/2282#issuecomment-993895683


   @jnturton , thanks for pushing the EVF V2 stuff forward! The `EasyFormatPlugin` should contain "shims" for the original format, for EVF1 and for EVF2. Given that you said you can't find it, and that the CSV reader is still based on V1, I wonder if there is some branch that never got pushed a PR? I'll do some research to determine what's what.


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

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

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



[GitHub] [drill] dzamo commented on pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
dzamo commented on pull request #2282:
URL: https://github.com/apache/drill/pull/2282#issuecomment-983644794


   Dear PR author and reviewers.
   
   This is a generic message to say that we would like to merge this PR in time for the 1.20 release.  Currently we're targeting a master branch freeze date of 2021-12-10 (10 Dec).  Please strive to complete development and review by this time, or indicate that the PR will need more time (and how much).
   
   Thank you.


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

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

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



[GitHub] [drill] lgtm-com[bot] commented on pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #2282:
URL: https://github.com/apache/drill/pull/2282#issuecomment-952362834


   This pull request **introduces 2 alerts** when merging dc60d28e52d997c272105b8eef96e0d9b06af73e into b6da35ece5a278a5ca72ecc33bafc98ba8f861f6 - [view on LGTM.com](https://lgtm.com/projects/g/apache/drill/rev/pr-5a63fcd28e1d8a3feb1bb69eae2701d56a187ca5)
   
   **new alerts:**
   
   * 2 for Unused format argument


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

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

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



[GitHub] [drill] cgivre commented on a change in pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
cgivre commented on a change in pull request #2282:
URL: https://github.com/apache/drill/pull/2282#discussion_r740503925



##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator> {

Review comment:
       @paul-rogers Is there an example somewhere of how to use the rowset loader to set up the schema automatically?  Is this as simple as checking to see whether the schema is provided and if so, use that?




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

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

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



[GitHub] [drill] dzamo commented on a change in pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
dzamo commented on a change in pull request #2282:
URL: https://github.com/apache/drill/pull/2282#discussion_r742913804



##########
File path: contrib/format-fixedwidth/src/main/java/org/apache/drill/exec/store/fixedwidth/FixedwidthBatchReader.java
##########
@@ -0,0 +1,200 @@
+/*
+ * 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.fixedwidth;
+
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.math.BigDecimal;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Locale;
+
+public class FixedwidthBatchReader implements ManagedReader<FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(FixedwidthBatchReader.class);
+  private FileSplit split;
+  private final int maxRecords;
+  private final FixedwidthFormatConfig config;
+  private CustomErrorContext errorContext;
+  private InputStream fsStream;
+  private ResultSetLoader loader;
+  private RowSetLoader writer;
+  private BufferedReader reader;
+  private int lineNum;
+
+  public FixedwidthBatchReader(FixedwidthFormatConfig config, int maxRecords) {
+    this.config = config;
+    this.maxRecords = maxRecords;
+  }
+
+
+  @Override
+  public boolean open(FileSchemaNegotiator negotiator) {
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    lineNum = 0;
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      negotiator.tableSchema(buildSchema(), true);
+      loader = negotiator.build();
+    } catch (Exception e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to open input file: {}", split.getPath().toString())
+        .addContext(errorContext)
+        .addContext(e.getMessage())
+        .build(logger);
+    }
+    reader = new BufferedReader(new InputStreamReader(fsStream, Charsets.UTF_8));
+    return true;
+  }
+
+  @Override
+  public boolean next() { // Use loader to read data from file to turn into Drill rows
+    String line;
+    RowSetLoader writer = loader.writer();
+
+    try {
+      line = reader.readLine();
+      while (!writer.isFull() && line != null) {
+        writer.start();
+        parseLine(line, writer);
+        writer.save();
+        line = reader.readLine();
+        lineNum++;
+      }
+    } catch (IOException e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to read input file: {}", split.getPath().toString())
+        .addContext(errorContext)
+        .addContext(e.getMessage())
+        .addContext("Line Number", lineNum)
+        .build(logger);
+    }
+    return writer.limitReached(maxRecords);  // returns false when maxRecords limit has been reached
+  }
+
+  @Override
+  public void close() {
+    if (fsStream != null){
+      AutoCloseables.closeSilently(fsStream);
+      fsStream = null;
+    }
+  }
+
+  private TupleMetadata buildSchema() {
+    SchemaBuilder builder = new SchemaBuilder();
+    for (FixedwidthFieldConfig field : config.getFields()) {
+      if (field.getType() == TypeProtos.MinorType.VARDECIMAL){
+        builder.addNullable(field.getName(), TypeProtos.MinorType.VARDECIMAL,38,4);
+        //revisit this
+      } else {
+        builder.addNullable(field.getName(), field.getType());
+      }
+    }
+    return builder.buildSchema();
+  }
+
+
+  private boolean parseLine(String line, RowSetLoader writer) throws IOException {
+    int i = 0;
+    TypeProtos.MinorType dataType;
+    String dateTimeFormat;
+    String value;
+    for (FixedwidthFieldConfig field : config.getFields()) {

Review comment:
       @cgivre @MFoss19 @estherbuchwalter here we are reading column data types from the format config, where we also specify their start and stop offsets.  But this format plugin can also accept data types from a provided schema.  So my question is: can we drop the data type information from the format config so that we don't introduce multiple ad-hoc ways of specifying this info?  This is genuinely a question because I don't know this subject well, but should we not work with data type specs here exactly the same way we do for CSV (cannot be provided in the format config I don't think)?




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

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

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



[GitHub] [drill] dzamo edited a comment on pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
dzamo edited a comment on pull request #2282:
URL: https://github.com/apache/drill/pull/2282#issuecomment-963159298


   > Let's consider a real world use case: some fixed width log generated by a database. Since the fields may be mashed together, there isn't a delimiter that you can use to divide the fields. You _could_ use however the logRegex reader to do this. That point aside for the moment, the way I imagined someone using this was that different configs could be set up and linked to workspaces such that if a file was in the `mysql_logs` folder, it would use the mysql log config, and if it was in the `postgres` it would use another.
   
   @cgivre  This use case would still work after two `CREATE SCHEMA` statements to set the names and data types, wouldn't it?  The schemas would be applied every subsequent query.
   
   > My opinion here is that the goal should be to get the cleanest data to the user as possible without the user having to rely on CASTs and other complicating factors.
   
   Let's drop the CASTs, those aren't fun.  So we're left with different ways a user can specify column names and types.
   
   1. With a `CREATE SCHEMA` against a directory.
   2. With an inline schema to a table function.
   3. With some plugin-specific format config that works for this plugin but not for others.
   
   Any one requires some effort, any one gets you to `select *` returning nice results (disclaimer: is this claim I'm making actually true?) which is super valuable.  So shouldn't we avoid the quirky 3 and commit to 1 and 2 consistently wherever we can?


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

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

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



[GitHub] [drill] dzamo commented on pull request #2282: DRILL-7978: Fixed Width Format Plugin

Posted by GitBox <gi...@apache.org>.
dzamo commented on pull request #2282:
URL: https://github.com/apache/drill/pull/2282#issuecomment-963159298


   > Let's consider a real world use case: some fixed width log generated by a database. Since the fields may be mashed together, there isn't a delimiter that you can use to divide the fields. You _could_ use however the logRegex reader to do this. That point aside for the moment, the way I imagined someone using this was that different configs could be set up and linked to workspaces such that if a file was in the `mysql_logs` folder, it would use the mysql log config, and if it was in the `postgres` it would use another.
   
   @cgivre  This use case would still work after two `CREATE SCHEMA` statements to set the names and data types, wouldn't it?  The schemas would be applied every subsequent query.
   
   > My opinion here is that the goal should be to get the cleanest data to the user as possible without the user having to rely on CASTs and other complicating factors.
   
   Let's drop the CASTs, those aren't fun.  So we're left with different ways a user can specify column names and types.
   
   1. With a `CREATE SCHEMA` against a directory.
   2. With an inline schema to a table function.
   3. With some plugin-specific format config that works for this plugin but not for others.
   
   Any one requires some effort, any one gets you to `select *` returning nice results (is this claim actually true) which is super valuable.  So shouldn't we avoid the quirky 3 and commit to 1 and 2 consistently wherever we can?


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

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

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