[GitHub] arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row set-based scan framework

2019-02-04 Thread GitBox
arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row 
set-based scan framework
URL: https://github.com/apache/drill/pull/1618#discussion_r253460539
 
 

 ##
 File path: 
exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestFileMetadataColumnParser.java
 ##
 @@ -154,93 +148,85 @@ public void testWildcard() {
 Lists.newArrayList(metadataManager.projectionParser()));
 
 List cols = scanProj.columns();
-assertEquals(7, cols.size());
+assertEquals(1, cols.size());
 assertEquals(UnresolvedColumn.WILDCARD, cols.get(0).nodeType());
-for (int i = 0; i < 4; i++) {
-  assertEquals(FileMetadataColumn.ID, cols.get(1+i).nodeType());
-}
-assertEquals(PartitionColumn.ID, cols.get(5).nodeType());
-assertEquals(PartitionColumn.ID, cols.get(6).nodeType());
   }
 
   /**
-   * Drill 1.1 - 1.11 and Drill 1.13 or later put metadata columns after
-   * data columns. Drill 1.12 moved them before data columns. For testing
-   * and compatibility, the client can request to use the Drill 1.12 position,
-   * though the after-data position is the default.
+   * Combine wildcard and file metadata columms. The wildcard expands
+   * table columns but not metadata columns.
*/
 
   @Test
-  public void testDrill1_12Wildcard() {
+  public void testWildcardAndFileMetaata() {
 Path filePath = new Path("hdfs:///w/x/y/z.csv");
 FileMetadataManager metadataManager = new FileMetadataManager(
-fixture.getOptionManager(), true,
+fixture.getOptionManager(),
 new Path("hdfs:///w"),
 Lists.newArrayList(filePath));
 
 ScanLevelProjection scanProj = new ScanLevelProjection(
-RowSetTestUtils.projectAll(),
-Lists.newArrayList(metadataManager.projectionParser()),
-true);
+RowSetTestUtils.projectList(
+SchemaPath.DYNAMIC_STAR,
+ScanTestUtils.FILE_NAME_COL,
+ScanTestUtils.SUFFIX_COL),
+Lists.newArrayList(metadataManager.projectionParser()));
 
 List cols = scanProj.columns();
-assertEquals(7, cols.size());
-for (int i = 0; i < 4; i++) {
-  assertEquals(FileMetadataColumn.ID, cols.get(i).nodeType());
-}
-assertEquals(PartitionColumn.ID, cols.get(4).nodeType());
-assertEquals(PartitionColumn.ID, cols.get(5).nodeType());
-assertEquals(UnresolvedColumn.WILDCARD, cols.get(6).nodeType());
+assertEquals(3, cols.size());
+assertEquals(UnresolvedColumn.WILDCARD, cols.get(0).nodeType());
+assertEquals(FileMetadataColumn.ID, cols.get(1).nodeType());
+assertEquals(FileMetadataColumn.ID, cols.get(2).nodeType());
   }
 
   /**
-   * Can't explicitly list file metadata columns with a wildcard in
-   * "legacy" mode: that is, when the wildcard already includes partition
-   * and file metadata columns.
+   * As above, but include implicit columns before and after the
+   * wildcard.
*/
 
   @Test
-  public void testErrorWildcardLegacyAndFileMetaata() {
-
+  public void testWildcardAndFileMetaataMixed() {
 
 Review comment:
   `Metaata` -> `Metadata`


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row set-based scan framework

2019-02-04 Thread GitBox
arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row 
set-based scan framework
URL: https://github.com/apache/drill/pull/1618#discussion_r253463714
 
 

 ##
 File path: 
exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestColumnsArray.java
 ##
 @@ -54,7 +54,7 @@ public void testColumnsArray() {
 
 Path filePath = new Path("hdfs:///w/x/y/z.csv");
 
 Review comment:
   Should not we use `file:///` in tests, not `hdfs:///`?


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row set-based scan framework

2019-02-04 Thread GitBox
arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row 
set-based scan framework
URL: https://github.com/apache/drill/pull/1618#discussion_r253463105
 
 

 ##
 File path: 
exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/RowSetTestUtils.java
 ##
 @@ -30,11 +29,9 @@ private RowSetTestUtils() { }
 
   public static List projectList(String... names) {
 List selected = new ArrayList<>();
-for (String name: names) {
-
-  // Parse from string does not handle wildcards.
-
-  if (name.equals(SchemaPath.DYNAMIC_STAR)) {
+for (String name : names) {
+  if (name.equals(SchemaPath.DYNAMIC_STAR) ||
+  name.equals("*")) {
 
 Review comment:
   As far as I understand start column is replaced with `**` in Calcite, do we 
need to check for `*` as well?


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row set-based scan framework

2019-02-04 Thread GitBox
arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row 
set-based scan framework
URL: https://github.com/apache/drill/pull/1618#discussion_r253460360
 
 

 ##
 File path: 
exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestFileMetadataColumnParser.java
 ##
 @@ -154,93 +148,85 @@ public void testWildcard() {
 Lists.newArrayList(metadataManager.projectionParser()));
 
 List cols = scanProj.columns();
-assertEquals(7, cols.size());
+assertEquals(1, cols.size());
 assertEquals(UnresolvedColumn.WILDCARD, cols.get(0).nodeType());
-for (int i = 0; i < 4; i++) {
-  assertEquals(FileMetadataColumn.ID, cols.get(1+i).nodeType());
-}
-assertEquals(PartitionColumn.ID, cols.get(5).nodeType());
-assertEquals(PartitionColumn.ID, cols.get(6).nodeType());
   }
 
   /**
-   * Drill 1.1 - 1.11 and Drill 1.13 or later put metadata columns after
-   * data columns. Drill 1.12 moved them before data columns. For testing
-   * and compatibility, the client can request to use the Drill 1.12 position,
-   * though the after-data position is the default.
+   * Combine wildcard and file metadata columms. The wildcard expands
+   * table columns but not metadata columns.
*/
 
   @Test
-  public void testDrill1_12Wildcard() {
+  public void testWildcardAndFileMetaata() {
 
 Review comment:
   `Metaata` -> `Metadata`


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row set-based scan framework

2019-02-04 Thread GitBox
arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row 
set-based scan framework
URL: https://github.com/apache/drill/pull/1618#discussion_r253459701
 
 

 ##
 File path: 
exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/ScanTestUtils.java
 ##
 @@ -104,4 +108,23 @@ public static TupleMetadata schema(ResolvedTuple output) {
 }
 return schema;
   }
+
+  public static List expandMetadata(int dirCount) {
+List selected = Lists.newArrayList(
+FULLY_QUALIFIED_NAME_COL,
+FILE_PATH_COL,
+FILE_NAME_COL,
+SUFFIX_COL);
+
+for (int i = 0; i < dirCount; i++) {
+  selected.add(PARTITION_COL + Integer.toString(i));
+}
+return RowSetTestUtils.projectList(selected);
+  }
+
+  public static List projectAllWithMetadata(int dirCount) {
+return RowSetTestUtils.concat(
+RowSetTestUtils.projectAll(),
+expandMetadata(2));
 
 Review comment:
   Maybe we should pass `dirCount` instead of `2`?


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row set-based scan framework

2019-02-04 Thread GitBox
arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row 
set-based scan framework
URL: https://github.com/apache/drill/pull/1618#discussion_r253465368
 
 

 ##
 File path: 
exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/text/compliant/TestCsv.java
 ##
 @@ -151,12 +151,77 @@ private String makeStatement(String fileName) {
 return "SELECT * FROM `dfs.data`.`" + fileName + "`";
   }
 
-  private void buildFile(String fileName, String[] data) throws IOException {
+  private static void buildFile(String fileName, String[] data) throws 
IOException {
 try(PrintWriter out = new PrintWriter(new FileWriter(new File(testDir, 
fileName {
   for (String line : data) {
 out.println(line);
   }
 }
   }
 
+  /**
+   * Verify that the wildcard expands columns to the header names, including
+   * case
+   */
+  @Test
+  public void testWildcard() throws IOException {
+String sql = "SELECT * FROM `dfs.data`.`%s`";
+RowSet actual = client.queryBuilder().sql(sql, CASE2_FILE_NAME).rowSet();
+
+BatchSchema expectedSchema = new SchemaBuilder()
+.add("a", MinorType.VARCHAR)
+.add("b", MinorType.VARCHAR)
+.add("c", MinorType.VARCHAR)
+.build();
+assertTrue(expectedSchema.isEquivalent(actual.batchSchema()));
+
+RowSet expected = new RowSetBuilder(client.allocator(), expectedSchema)
+.addRow("10", "foo", "bar")
+.build();
+RowSetUtilities.verify(expected, actual);
+  }
+
+  /**
+   * Verify that implicit columns are recognized and populated. Sanity test
+   * of just one implicit column.
+   */
+  @Test
+  public void testImplicitColsExplicitSelect() throws IOException {
+String sql = "SELECT A, filename FROM `dfs.data`.`%s`";
+RowSet actual = client.queryBuilder().sql(sql, CASE2_FILE_NAME).rowSet();
+
+BatchSchema expectedSchema = new SchemaBuilder()
+.add("A", MinorType.VARCHAR)
+.addNullable("filename", MinorType.VARCHAR)
 
 Review comment:
   Why we add implicit file column as nullable? Should be it be required?


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row set-based scan framework

2019-01-28 Thread GitBox
arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row 
set-based scan framework
URL: https://github.com/apache/drill/pull/1618#discussion_r251458228
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileMetadataManager.java
 ##
 @@ -0,0 +1,239 @@
+/*
+ * 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.physical.impl.scan.file;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.directory.api.util.Strings;
+import org.apache.drill.common.map.CaseInsensitiveMap;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.physical.impl.scan.project.ColumnProjection;
+import org.apache.drill.exec.physical.impl.scan.project.ConstantColumnLoader;
+import org.apache.drill.exec.physical.impl.scan.project.MetadataManager;
+import org.apache.drill.exec.physical.impl.scan.project.ResolvedTuple;
+import 
org.apache.drill.exec.physical.impl.scan.project.ScanLevelProjection.ScanProjectionParser;
+import 
org.apache.drill.exec.physical.impl.scan.project.SchemaLevelProjection.SchemaProjectionResolver;
+import org.apache.drill.exec.physical.impl.scan.project.VectorSource;
+import org.apache.drill.exec.physical.rowSet.ResultVectorCache;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.server.options.OptionSet;
+import org.apache.drill.exec.store.ColumnExplorer.ImplicitFileColumns;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.hadoop.fs.Path;
+
+import 
org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTesting;
+
+public class FileMetadataManager implements MetadataManager, 
SchemaProjectionResolver, VectorSource {
+
+  // Input
+
+  private Path scanRootDir;
+  private FileMetadata currentFile;
+
+  // Config
+
+  protected final String partitionDesignator;
+  protected List implicitColDefns = new ArrayList<>();
+  protected Map fileMetadataColIndex = 
CaseInsensitiveMap.newHashMap();
+  protected boolean useLegacyWildcardExpansion = true;
 
 Review comment:
   Looked at `org.apache.drill.exec.physical.impl.ScanBatch` class, when table 
has partition columns they are added before the table columns, implicit file 
columns are added only if they are present in the select. If they are, they are 
added before partition columns.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row set-based scan framework

2019-01-28 Thread GitBox
arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row 
set-based scan framework
URL: https://github.com/apache/drill/pull/1618#discussion_r251437978
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileMetadataColumnsParser.java
 ##
 @@ -0,0 +1,166 @@
+/*
+ * 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.physical.impl.scan.file;
+
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.drill.exec.physical.impl.scan.project.ColumnProjection;
+import org.apache.drill.exec.physical.impl.scan.project.ScanLevelProjection;
+import 
org.apache.drill.exec.physical.impl.scan.project.ScanLevelProjection.ScanProjectionParser;
+import 
org.apache.drill.exec.physical.rowSet.project.RequestedTuple.RequestedColumn;
+
+public class FileMetadataColumnsParser implements ScanProjectionParser {
+
+  static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(FileMetadataColumnsParser.class);
+
+  // Internal
+
+  private final FileMetadataManager metadataManager;
+  private final Pattern partitionPattern;
+  private ScanLevelProjection builder;
+
+  // Output
+
+  boolean hasMetadata;
+
+  public FileMetadataColumnsParser(FileMetadataManager metadataManager) {
+this.metadataManager = metadataManager;
+partitionPattern = Pattern.compile(metadataManager.partitionDesignator + 
"(\\d+)", Pattern.CASE_INSENSITIVE);
+  }
+
+  @Override
+  public void bind(ScanLevelProjection builder) {
+this.builder = builder;
+  }
+
+  @Override
+  public boolean parse(RequestedColumn inCol) {
+Matcher m = partitionPattern.matcher(inCol.name());
+if (m.matches()) {
+  return buildPartitionColumn(m, inCol);
+}
+
+FileMetadataColumnDefn defn = 
metadataManager.fileMetadataColIndex.get(inCol.name());
+if (defn != null) {
+  return buildMetadataColumn(defn, inCol);
+}
+if (inCol.isWildcard()) {
+  buildWildcard();
+
+  // Don't consider this a match.
+}
+return false;
+  }
+
+  private boolean buildPartitionColumn(Matcher m, RequestedColumn inCol) {
+
+// If the projected column is a map or array, then it shadows the
+// partition column. Example: dir0.x, dir0[2].
+
+if (! inCol.isSimple()) {
+  logger.warn("Partition column {} is shadowed by a projected {}",
+  inCol.name(), inCol.summary());
+  return false;
+}
+if (builder.hasWildcard()) {
+  wildcardAndMetadataError();
+}
+
+// Partition column
+
+builder.addMetadataColumn(
+new PartitionColumn(
+  inCol.name(),
+  Integer.parseInt(m.group(1;
+hasMetadata = true;
+return true;
+  }
+
+  private boolean buildMetadataColumn(FileMetadataColumnDefn defn,
+  RequestedColumn inCol) {
+
+// If the projected column is a map or array, then it shadows the
+// metadata column. Example: filename.x, filename[2].
+
+if (! inCol.isSimple()) {
+  logger.warn("File metadata column {} is shadowed by a projected {}",
+  inCol.name(), inCol.summary());
+  return false;
+}
+if (builder.hasWildcard()) {
+  wildcardAndMetadataError();
+}
+
+// File metadata (implicit) column
+
+builder.addMetadataColumn(new FileMetadataColumn(inCol.name(), defn));
+hasMetadata = true;
+return true;
+  }
+
+  private void buildWildcard() {
+if (hasMetadata) {
+  wildcardAndMetadataError();
+}
+if (metadataManager.useLegacyWildcardExpansion) {
+
+  // Star column: this is a SELECT * query.
+
+  // Old-style wildcard handling inserts all metadata columns in
+  // the scanner, removes them in Project.
+  // Fill in the file metadata columns. Can do here because the
+  // set is constant across all files.
+
+  expandWildcard();
+  hasMetadata = true;
+}
+  }
+
+  protected void expandWildcard() {
+
+// Legacy wildcard expansion: include the file metadata and
+// file partitions for this file.
+// This is a disadvantage for a * query: files at different directory
+// levels will have different numbers of columns. Would be better to
+// return this data 

[GitHub] arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row set-based scan framework

2019-01-28 Thread GitBox
arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row 
set-based scan framework
URL: https://github.com/apache/drill/pull/1618#discussion_r251437169
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/ScanOperatorExec.java
 ##
 @@ -0,0 +1,278 @@
+/*
+ * 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.physical.impl.scan;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.physical.impl.protocol.BatchAccessor;
+import org.apache.drill.exec.physical.impl.protocol.OperatorExec;
+import org.apache.drill.exec.physical.impl.protocol.VectorContainerAccessor;
+
+import 
org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Implementation of the revised scan operator that uses a mutator aware of
+ * batch sizes. This is the successor to {@link ScanBatch} and should be used
+ * by all new scan implementations.
+ *
+ * Scanner Framework
+ *
+ * Acts as an adapter between the operator protocol and the row reader
+ * protocol.
+ * 
+ * The scan operator itself is simply a framework for handling a set of 
readers;
+ * it knows nothing other than the interfaces of the components it works with;
+ * delegating all knowledge of schemas, projection, reading and the like to
+ * implementations of those interfaces. Because that work is complex, a set
+ * of frameworks exist to handle most common use cases, but a specialized 
reader
+ * can create a framework or reader from scratch.
+ * 
+ * Error handling in this class is minimal: the enclosing record batch iterator
+ * is responsible for handling exceptions. Error handling relies on the fact
+ * that the iterator will call close() regardless of which exceptions
+ * are thrown.
+ *
+ * Protocol
+ *
+ * The scanner works directly with two other interfaces
+ * 
+ * The {@link ScanOperatorEvents} implementation provides the set of readers to
+ * use. This class can simply maintain a list, or can create the reader on
+ * demand.
+ * 
+ * More subtly, the factory also handles projection issues and manages vectors
+ * across subsequent readers. A number of factories are available for the most
+ * common cases. Extend these to implement a version specific to a data source.
+ * 
+ * The {@link RowBatchReader} is a surprisingly minimal interface that
+ * nonetheless captures the essence of reading a result set as a set of 
batches.
+ * The factory implementations mentioned above implement this interface to 
provide
+ * commonly-used services, the most important of which is access to a
+ * {#link ResultSetLoader} to write values into value vectors.
+ *
+ * Schema Versions
+ * Readers may change schemas from time to time. To track such changes,
+ * this implementation tracks a batch schema version, maintained by comparing
+ * one schema with the next.
+ * 
+ * Readers can discover columns as they read data, such as with any JSON-based
+ * format. In this case, the row set mutator also provides a schema version,
+ * but a fine-grained one that changes each time a column is added.
+ * 
+ * The two schema versions serve different purposes and are not 
interchangeable.
+ * For example, if a scan reads two files, both will build up their own 
schemas,
+ * each increasing its internal version number as work proceeds. But, at the
+ * end of each batch, the schemas may (and, in fact, should) be identical,
+ * which is the schema version downstream operators care about.
+ */
+
+public class ScanOperatorExec implements OperatorExec {
+
+  private enum State { START, READER, END, FAILED, CLOSED }
+
+  static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(ScanOperatorExec.class);
+
+  private final ScanOperatorEvents factory;
+  protected final VectorContainerAccessor containerAccessor = new 
VectorContainerAccessor();
+  private State state = State.START;
+  protected OperatorContext context;
+  private int readerCount;
+  private ReaderState readerState;
+
+  public ScanOperatorExec(ScanOperatorEvents factory) {
+this.factory = 

[GitHub] arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row set-based scan framework

2019-01-24 Thread GitBox
arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row 
set-based scan framework
URL: https://github.com/apache/drill/pull/1618#discussion_r250598167
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/BaseFileScanFramework.java
 ##
 @@ -0,0 +1,177 @@
+/*
+ * 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.physical.impl.scan.file;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.impl.scan.RowBatchReader;
+import 
org.apache.drill.exec.physical.impl.scan.framework.AbstractScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.impl.scan.framework.SchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ShimBatchReader;
+import org.apache.drill.exec.store.dfs.DrillFileSystem;
+import org.apache.drill.exec.store.dfs.easy.FileWork;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileSplit;
+
+/**
+ * Scan framework for a file that implements metadata columns (AKA "implicit"
+ * columns and partition columns.)
+ * 
+ * Framework iterators over file descriptions, creating readers at the
+ * moment they are needed. This allows simpler logic because, at the point of
+ * reader creation, we have a file system, context and so on.
+ */
+
+public abstract class BaseFileScanFramework
+extends AbstractScanFramework {
+
+  static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(BaseFileScanFramework.class);
+
+  /**
+   * The file schema negotiator adds no behavior at present, but is
+   * created as a placeholder anticipating the need for file-specific
+   * behavior later.
+   */
+
+  public interface FileSchemaNegotiator extends SchemaNegotiator {
+  }
+
+  private final List files;
+  private final Configuration fsConfig;
+  private List spilts = new ArrayList<>();
+  private Iterator splitIter;
+  private Path scanRootDir;
+  private boolean useLegacyWildcardExpansion = true;
+  protected DrillFileSystem dfs;
+  private FileMetadataManager metadataManager;
+
+  public BaseFileScanFramework(List projection,
+  List files,
+  Configuration fsConf) {
+super(projection);
+this.files = files;
+this.fsConfig = fsConf;
+  }
+
+  /**
+   * Specify the selection root for a directory scan, if any.
+   * Used to populate partition columns.
+   * @param rootPath Hadoop file path for the directory
+   */
+
+  public void setSelectionRoot(Path rootPath) {
+this.scanRootDir = rootPath;
+  }
+
+  /**
+   * For historical reasons, Drill adds all metadata columns for a wildcard
+   * query. The project operator then drops those that are not needed. This
+   * flag disables that behavior if newer versions of Drill don't need the
+   * "create-then-delete" behavior.
+   *
+   * @param flag true to use the old-style expansion, false to not include
+   * metadata columns when expanding the wildcard column
+   */
+
+  public void useLegacyWildcardExpansion(boolean flag) {
+useLegacyWildcardExpansion = flag;
+  }
+
+  @Override
+  protected void configure() {
+super.configure();
+
+// Create the Drill file system.
+
+try {
+  dfs = context.newFileSystem(fsConfig);
+} catch (IOException e) {
+  throw UserException.dataReadError(e)
+.addContext("Failed to create FileSystem")
+.build(logger);
+}
+
+// Prepare the list of files. We need the list of paths up
+// front to compute the maximum partition. Then, we need to
+// iterate over the splits to create readers on demand.
+
+List paths = new ArrayList<>();
+for(FileWork work : files) {
 
 Review comment:
   Please add space after `for` before `(`.


This is an automated message from the Apache Git Service.
To respond to 

[GitHub] arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row set-based scan framework

2019-01-24 Thread GitBox
arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row 
set-based scan framework
URL: https://github.com/apache/drill/pull/1618#discussion_r250608895
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileMetadataColumn.java
 ##
 @@ -0,0 +1,63 @@
+/*
+ * 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.physical.impl.scan.file;
+
+import org.apache.drill.exec.physical.impl.scan.project.VectorSource;
+
+public class FileMetadataColumn extends MetadataColumn {
+
+  public static final int ID = 15;
+
+  private final FileMetadataColumnDefn defn;
+
+  /**
+   * Constructor for resolved column.
+   *
+   * @param name
 
 Review comment:
   Please add params descriptions.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row set-based scan framework

2019-01-24 Thread GitBox
arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row 
set-based scan framework
URL: https://github.com/apache/drill/pull/1618#discussion_r250608930
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileMetadataColumn.java
 ##
 @@ -0,0 +1,63 @@
+/*
+ * 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.physical.impl.scan.file;
+
+import org.apache.drill.exec.physical.impl.scan.project.VectorSource;
+
+public class FileMetadataColumn extends MetadataColumn {
+
+  public static final int ID = 15;
+
+  private final FileMetadataColumnDefn defn;
+
+  /**
+   * Constructor for resolved column.
+   *
+   * @param name
+   * @param defn
+   * @param fileInfo
+   * @param projection
+   */
+  public FileMetadataColumn(String name, FileMetadataColumnDefn defn,
+  FileMetadata fileInfo, VectorSource source, int sourceIndex) {
+super(name, defn.dataType(), defn.defn.getValue(fileInfo.filePath()), 
source, sourceIndex);
+this.defn = defn;
+  }
+
+  /**
+   * Constructor for unresolved column.
+   *
+   * @param name
 
 Review comment:
   Same here.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row set-based scan framework

2019-01-24 Thread GitBox
arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row 
set-based scan framework
URL: https://github.com/apache/drill/pull/1618#discussion_r250626306
 
 

 ##
 File path: 
exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanBatchWriters.java
 ##
 @@ -0,0 +1,122 @@
+/*
+ * 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.
+ 
**/
 
 Review comment:
   Please replace with `*/`.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row set-based scan framework

2019-01-24 Thread GitBox
arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row 
set-based scan framework
URL: https://github.com/apache/drill/pull/1618#discussion_r250597646
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/BaseFileScanFramework.java
 ##
 @@ -0,0 +1,177 @@
+/*
+ * 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.physical.impl.scan.file;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.impl.scan.RowBatchReader;
+import 
org.apache.drill.exec.physical.impl.scan.framework.AbstractScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.impl.scan.framework.SchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ShimBatchReader;
+import org.apache.drill.exec.store.dfs.DrillFileSystem;
+import org.apache.drill.exec.store.dfs.easy.FileWork;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileSplit;
+
+/**
+ * Scan framework for a file that implements metadata columns (AKA "implicit"
+ * columns and partition columns.)
+ * 
+ * Framework iterators over file descriptions, creating readers at the
+ * moment they are needed. This allows simpler logic because, at the point of
+ * reader creation, we have a file system, context and so on.
+ */
+
+public abstract class BaseFileScanFramework
+extends AbstractScanFramework {
+
+  static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(BaseFileScanFramework.class);
+
+  /**
+   * The file schema negotiator adds no behavior at present, but is
+   * created as a placeholder anticipating the need for file-specific
+   * behavior later.
 
 Review comment:
   Can you please give an example?


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row set-based scan framework

2019-01-24 Thread GitBox
arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row 
set-based scan framework
URL: https://github.com/apache/drill/pull/1618#discussion_r250586889
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/ReaderState.java
 ##
 @@ -0,0 +1,287 @@
+/*
+ * 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.physical.impl.scan;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.record.VectorContainer;
+
+/**
+ * Manages a row batch reader through its lifecycle. Created when the reader
+ * is opened, discarded when the reader is closed. Encapsulates state that
+ * follows the life of the reader. This moves common scanner state out of
+ * each reader in order to make the reader as simple as possible.
+ * 
+ * This class is private to the scan operator and is not meant to be used,
+ * or even visible, outside of that operator itself. Instead, all 
reader-specific
+ * functionality should be in the {@link RowBatchReader} subclass.
+ */
+
+class ReaderState {
+  static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(ReaderState.class);
+
+  private enum State { START, LOOK_AHEAD, LOOK_AHEAD_WITH_EOF, ACTIVE, EOF, 
CLOSED };
+
+  final ScanOperatorExec scanOp;
+  private final RowBatchReader reader;
+  private State state = State.START;
+  private VectorContainer lookahead;
+  private int schemaVersion = -1;
+
+  public ReaderState(ScanOperatorExec scanOp, RowBatchReader reader) {
+this.scanOp = scanOp;
+this.reader = reader;
+  }
+
+  /**
+   * Open the next available reader, if any, preparing both the
+   * reader and table loader.
+   * @return true if another reader is active, false if no more
+   * readers are available
+   */
+
+  boolean open() {
+
+// Open the reader. This can fail. if it does, clean up.
+
+try {
+
+  // The reader can return a "soft" failure: the open worked, but
+  // the file is empty, non-existent or some other form of "no data."
+  // Handle this by immediately moving to EOF. The scanner will quietly
+  // pass over this reader and move onto the next, if any.
+
+  if (! reader.open()) {
+state = State.EOF;
+return false;
+  }
+
+// When catching errors, leave the reader member set;
+// we must close it on close() later.
+
+} catch (UserException e) {
+
+  // Throw user exceptions as-is
+
+  throw e;
+} catch (Throwable t) {
+
+  // Wrap all others in a user exception.
+
+  throw UserException.executionError(t)
+.addContext("Open failed for reader", reader.name())
+.build(logger);
+}
+
+state = State.ACTIVE;
+return true;
+  }
+
+  /**
+   * Prepare the schema for this reader. Called for the first reader within a
+   * scan batch, if the reader returns true from open(). If
+   * this is an early-schema reader, then the result set loader already has
+   * the proper value vectors set up. If this is a late-schema reader, we must
+   * read one batch to get the schema, then set aside the data for the next
+   * call to next().
+   * 
+   * Semantics for all readers:
+   * 
+   * If the file was not found, open() returned false and this
+   * method should never be called.
+   * 
+   * 
+   * Semantics for early-schema readers:
+   * 
+   * If if turned out that the file was
+   * empty when trying to read the schema, open() returned false
+   * and this method should never be called.
+   * Otherwise, if a schema was available, then the schema is already
+   * set up in the result set loader as the result of schema negotiation, and
+   * this method simply returns true.
+   * 
+   * 
+   * Semantics for late-schema readers:
+   * 
+   * This method will ask the reader to
+   * read a batch. If the reader hits EOF before finding any data, this method
+   * will return false, indicating that no schema is available.
+   * If the reader can read enough of the file to
+   * figure out the schema, but the file has no data, then this method will
+   * return true and a schema will be available. The first call to
+   * next() will report EOF.
+   * Otherwise, this method returns 

[GitHub] arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row set-based scan framework

2019-01-24 Thread GitBox
arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row 
set-based scan framework
URL: https://github.com/apache/drill/pull/1618#discussion_r250608494
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileMetadata.java
 ##
 @@ -0,0 +1,90 @@
+/*
+ * 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.physical.impl.scan.file;
+
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * Specify the file name and optional selection root. If the selection root
+ * is provided, then partitions are defined as the portion of the file name
+ * that is not also part of the selection root. That is, if selection root is
+ * /a/b and the file path is /a/b/c/d.csv, then dir0 is c.
+ */
+
+public class FileMetadata {
+
+  private final Path filePath;
+  private final String[] dirPath;
+
+  public FileMetadata(Path filePath, Path selectionRoot) {
+this.filePath = filePath;
+
+// If the data source is not a file, no file metadata is available.
+
+if (selectionRoot == null || filePath == null) {
+  dirPath = null;
+  return;
+}
+
+// If the query is against a single file, selection root and file path
+// will be identical, oddly.
+
+Path rootPath = Path.getPathWithoutSchemeAndAuthority(selectionRoot);
+Path bareFilePath = Path.getPathWithoutSchemeAndAuthority(filePath);
+if (rootPath.equals(bareFilePath)) {
+  dirPath = null;
+  return;
+}
+
+// Result of splitting /x/y is ["", "x", "y"], so ignore first.
+
+String[] r = rootPath.toString().split("/");
 
 Review comment:
   I think it's more preferable to use `Path` methods to extract partitions 
rather than splitting.
   Please see `listPartitionValues` as example 
https://github.com/apache/drill/blob/8edeb49873d1a1710cfe28e0b49364d07eb1aef4/exec/java-exec/src/main/java/org/apache/drill/exec/store/ColumnExplorer.java#L200


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row set-based scan framework

2019-01-24 Thread GitBox
arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row 
set-based scan framework
URL: https://github.com/apache/drill/pull/1618#discussion_r250591780
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/ScanOperatorExec.java
 ##
 @@ -0,0 +1,278 @@
+/*
+ * 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.physical.impl.scan;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.physical.impl.protocol.BatchAccessor;
+import org.apache.drill.exec.physical.impl.protocol.OperatorExec;
+import org.apache.drill.exec.physical.impl.protocol.VectorContainerAccessor;
+
+import 
org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Implementation of the revised scan operator that uses a mutator aware of
+ * batch sizes. This is the successor to {@link ScanBatch} and should be used
+ * by all new scan implementations.
+ *
+ * Scanner Framework
+ *
+ * Acts as an adapter between the operator protocol and the row reader
+ * protocol.
+ * 
+ * The scan operator itself is simply a framework for handling a set of 
readers;
+ * it knows nothing other than the interfaces of the components it works with;
+ * delegating all knowledge of schemas, projection, reading and the like to
+ * implementations of those interfaces. Because that work is complex, a set
+ * of frameworks exist to handle most common use cases, but a specialized 
reader
+ * can create a framework or reader from scratch.
+ * 
+ * Error handling in this class is minimal: the enclosing record batch iterator
+ * is responsible for handling exceptions. Error handling relies on the fact
+ * that the iterator will call close() regardless of which exceptions
+ * are thrown.
+ *
+ * Protocol
+ *
+ * The scanner works directly with two other interfaces
+ * 
+ * The {@link ScanOperatorEvents} implementation provides the set of readers to
+ * use. This class can simply maintain a list, or can create the reader on
+ * demand.
+ * 
+ * More subtly, the factory also handles projection issues and manages vectors
+ * across subsequent readers. A number of factories are available for the most
+ * common cases. Extend these to implement a version specific to a data source.
+ * 
+ * The {@link RowBatchReader} is a surprisingly minimal interface that
+ * nonetheless captures the essence of reading a result set as a set of 
batches.
+ * The factory implementations mentioned above implement this interface to 
provide
+ * commonly-used services, the most important of which is access to a
+ * {#link ResultSetLoader} to write values into value vectors.
+ *
+ * Schema Versions
+ * Readers may change schemas from time to time. To track such changes,
+ * this implementation tracks a batch schema version, maintained by comparing
+ * one schema with the next.
+ * 
+ * Readers can discover columns as they read data, such as with any JSON-based
+ * format. In this case, the row set mutator also provides a schema version,
+ * but a fine-grained one that changes each time a column is added.
+ * 
+ * The two schema versions serve different purposes and are not 
interchangeable.
+ * For example, if a scan reads two files, both will build up their own 
schemas,
+ * each increasing its internal version number as work proceeds. But, at the
+ * end of each batch, the schemas may (and, in fact, should) be identical,
+ * which is the schema version downstream operators care about.
+ */
+
+public class ScanOperatorExec implements OperatorExec {
+
+  private enum State { START, READER, END, FAILED, CLOSED }
+
+  static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(ScanOperatorExec.class);
+
+  private final ScanOperatorEvents factory;
+  protected final VectorContainerAccessor containerAccessor = new 
VectorContainerAccessor();
+  private State state = State.START;
+  protected OperatorContext context;
+  private int readerCount;
+  private ReaderState readerState;
+
+  public ScanOperatorExec(ScanOperatorEvents factory) {
+this.factory = 

[GitHub] arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row set-based scan framework

2019-01-24 Thread GitBox
arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row 
set-based scan framework
URL: https://github.com/apache/drill/pull/1618#discussion_r250591467
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/ScanOperatorExec.java
 ##
 @@ -0,0 +1,278 @@
+/*
+ * 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.physical.impl.scan;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.physical.impl.protocol.BatchAccessor;
+import org.apache.drill.exec.physical.impl.protocol.OperatorExec;
+import org.apache.drill.exec.physical.impl.protocol.VectorContainerAccessor;
+
+import 
org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Implementation of the revised scan operator that uses a mutator aware of
+ * batch sizes. This is the successor to {@link ScanBatch} and should be used
+ * by all new scan implementations.
+ *
+ * Scanner Framework
+ *
+ * Acts as an adapter between the operator protocol and the row reader
+ * protocol.
+ * 
+ * The scan operator itself is simply a framework for handling a set of 
readers;
+ * it knows nothing other than the interfaces of the components it works with;
+ * delegating all knowledge of schemas, projection, reading and the like to
+ * implementations of those interfaces. Because that work is complex, a set
+ * of frameworks exist to handle most common use cases, but a specialized 
reader
+ * can create a framework or reader from scratch.
+ * 
+ * Error handling in this class is minimal: the enclosing record batch iterator
+ * is responsible for handling exceptions. Error handling relies on the fact
+ * that the iterator will call close() regardless of which exceptions
+ * are thrown.
+ *
+ * Protocol
+ *
+ * The scanner works directly with two other interfaces
+ * 
+ * The {@link ScanOperatorEvents} implementation provides the set of readers to
+ * use. This class can simply maintain a list, or can create the reader on
+ * demand.
+ * 
+ * More subtly, the factory also handles projection issues and manages vectors
+ * across subsequent readers. A number of factories are available for the most
+ * common cases. Extend these to implement a version specific to a data source.
+ * 
+ * The {@link RowBatchReader} is a surprisingly minimal interface that
+ * nonetheless captures the essence of reading a result set as a set of 
batches.
+ * The factory implementations mentioned above implement this interface to 
provide
+ * commonly-used services, the most important of which is access to a
+ * {#link ResultSetLoader} to write values into value vectors.
+ *
+ * Schema Versions
+ * Readers may change schemas from time to time. To track such changes,
+ * this implementation tracks a batch schema version, maintained by comparing
+ * one schema with the next.
+ * 
+ * Readers can discover columns as they read data, such as with any JSON-based
+ * format. In this case, the row set mutator also provides a schema version,
+ * but a fine-grained one that changes each time a column is added.
+ * 
+ * The two schema versions serve different purposes and are not 
interchangeable.
+ * For example, if a scan reads two files, both will build up their own 
schemas,
+ * each increasing its internal version number as work proceeds. But, at the
+ * end of each batch, the schemas may (and, in fact, should) be identical,
+ * which is the schema version downstream operators care about.
+ */
+
+public class ScanOperatorExec implements OperatorExec {
+
+  private enum State { START, READER, END, FAILED, CLOSED }
+
+  static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(ScanOperatorExec.class);
+
+  private final ScanOperatorEvents factory;
+  protected final VectorContainerAccessor containerAccessor = new 
VectorContainerAccessor();
+  private State state = State.START;
+  protected OperatorContext context;
+  private int readerCount;
+  private ReaderState readerState;
+
+  public ScanOperatorExec(ScanOperatorEvents factory) {
+this.factory = 

[GitHub] arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row set-based scan framework

2019-01-24 Thread GitBox
arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row 
set-based scan framework
URL: https://github.com/apache/drill/pull/1618#discussion_r250618703
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/SchemaNegotiator.java
 ##
 @@ -0,0 +1,108 @@
+/*
+ * 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.physical.impl.scan.framework;
+
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+
+/**
+ * Negotiates the table schema with the scanner framework and provides
+ * context information for the reader. In a typical scan, the physical
+ * plan provides the project list: the set of columns that the query
+ * expects. Readers provide a table schema: the set of columns actually
+ * available. The scan framework combines the two lists to determine
+ * the available table columns that must be read, along with any additional
+ * to be added. Additional columns can be file metadata (if the storage
+ * plugin requests them), or can be null columns added for projected
+ * columns that don't actually exist in the table.
+ * 
+ * The reader provides the table schema in one of two ways:
+ * 
+ * If the reader is of "early schema" type, then the reader calls
+ * {@link #setTableSchema(TupleMetadata)} to provide that schema.
+ * If the reader is of "late schema" type, then the reader discovers
+ * the schema as the data is read, calling the
+ * {@link RowSetLoader#addColumn()} method to add each column as it is
+ * discovered.
+ * 
+ * Either way, the project list from the physical plan determines which
+ * table columns are materialized and which are not. Readers are provided
+ * for all table columns for readers that must read sequentially, but
+ * only the materialized columns are written to value vectors.
+ * 
+ * Regardless of the schema type, the result of building the schema is a
+ * result set loader used to prepare batches for use in the query. The reader
+ * can simply read all columns, allowing the framework to discard unwanted
+ * values. Or for efficiency, the reader can check the column metadata to
+ * determine if a column is projected, and if not, then don't even read
+ * the column from the input source.
+ */
+
+public interface SchemaNegotiator {
+
+  OperatorContext context();
+
+  /**
+   * Specify the type of table schema. Required only in the obscure
+   * case of an early-schema table with an empty schema, else inferred.
+   * (Set to {@link TableSchemaType#EARLY} if no columns provided, or
+   * to {@link TableSchemaType#LATE if at least one column is provided.)
+   * @param type the table schema type
+   */
+
+  void setTableSchema(TupleMetadata schema);
 
 Review comment:
   If we set the schema upfront? Will we fail if schema is partial and contains 
information about only certain columns? Or unspecified columns types will be 
inferred?


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row set-based scan framework

2019-01-24 Thread GitBox
arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row 
set-based scan framework
URL: https://github.com/apache/drill/pull/1618#discussion_r250594821
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/ColumnsArrayParser.java
 ##
 @@ -0,0 +1,173 @@
+/*
+ * 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.physical.impl.scan.columns;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.physical.impl.scan.project.ColumnProjection;
+import org.apache.drill.exec.physical.impl.scan.project.ScanLevelProjection;
+import 
org.apache.drill.exec.physical.impl.scan.project.ScanLevelProjection.ScanProjectionParser;
+import org.apache.drill.exec.physical.impl.scan.project.UnresolvedColumn;
+import org.apache.drill.exec.physical.rowSet.project.RequestedColumnImpl;
+import 
org.apache.drill.exec.physical.rowSet.project.RequestedTuple.RequestedColumn;
+import org.apache.drill.exec.store.easy.text.compliant.RepeatedVarCharOutput;
+
+/**
+ * Parses the `columns` array. Doing so is surprisingly complex.
+ * 
+ * Depending on what is known about the input file, the `columns`
+ * array may be required or optional.
+ * If the columns array is required, then the wildcard (`*`)
+ * expands to `columns`.
+ * If the columns array appears, then no other table columns
+ * can appear.
+ * If the columns array appears, then the wildcard cannot also
+ * appear, unless that wildcard expanded to be `columns` as
+ * described above.
+ * The query can select specific elements such as `columns`[2].
+ * In this case, only array elements can appear, not the unindexed
+ * `columns` column.
+ * 
+ * 
+ * It falls to this parser to detect a not-uncommon user error, a
+ * query such as the following:
+ * SELECT max(columns[1]) AS col1
+ * FROM cp.`textinput/input1.csv`
+ * WHERE col1 IS NOT NULL
+ * 
+ * In standard SQL, column aliases are not allowed in the WHERE
+ * clause. So, Drill will push two columns down to the scan operator:
+ * `columns`[1] and `col1`. This parser will detect the "extra"
+ * columns and must provide a message that helps the user identify
+ * the likely original problem.
+ */
+
+public class ColumnsArrayParser implements ScanProjectionParser {
+  static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(ColumnsArrayParser.class);
+
+  // Config
+
+  private final boolean requireColumnsArray;
+
+  // Internals
+
+  private ScanLevelProjection builder;
+
+  // Output
+
+  private UnresolvedColumnsArrayColumn columnsArrayCol;
+
+  public ColumnsArrayParser(boolean requireColumnsArray) {
+this.requireColumnsArray = requireColumnsArray;
+  }
+
+  @Override
+  public void bind(ScanLevelProjection builder) {
+this.builder = builder;
+  }
+
+  @Override
+  public boolean parse(RequestedColumn inCol) {
+if (requireColumnsArray && inCol.isWildcard()) {
+  expandWildcard();
+  return true;
+}
+if (! inCol.nameEquals(ColumnsArrayManager.COLUMNS_COL)) {
+  return false;
+}
+
+// The columns column cannot be a map. That is, the following is
+// not allowed: columns.foo.
+
+if (inCol.isTuple()) {
+  throw UserException
+.validationError()
+.message("{} has mep elements, but cannot be a map", inCol.name())
 
 Review comment:
   mep -> map


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row set-based scan framework

2019-01-24 Thread GitBox
arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row 
set-based scan framework
URL: https://github.com/apache/drill/pull/1618#discussion_r250597030
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/ColumnsScanFramework.java
 ##
 @@ -0,0 +1,121 @@
+/*
+ * 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.physical.impl.scan.columns;
+
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.impl.scan.file.BaseFileScanFramework;
+import 
org.apache.drill.exec.physical.impl.scan.file.BaseFileScanFramework.FileSchemaNegotiator;
+import 
org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiatorImpl;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.impl.scan.framework.ShimBatchReader;
+import org.apache.drill.exec.store.dfs.DrillFileSystem;
+import org.apache.drill.exec.store.dfs.easy.FileWork;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.FileSplit;
+
+/**
+ * Scan framework for a file that supports the special "columns" column.
+ */
+
+public class ColumnsScanFramework extends 
BaseFileScanFramework {
+
+  public interface FileReaderCreator {
+ManagedReader makeBatchReader(
+DrillFileSystem dfs,
+FileSplit split) throws ExecutionSetupException;
+  }
+
+  /**
+   * Schema negotiator that supports the file scan options plus access
+   * to the specific selected columns indexes.
+   */
+
+  public interface ColumnsSchemaNegotiator extends FileSchemaNegotiator {
+boolean columnsArrayProjected();
+boolean[] projectedIndexes();
+  }
+
+  /**
+   * Implementation of the columns array schema negotiator.
+   */
+
+  public static class ColumnsSchemaNegotiatorImpl extends 
FileSchemaNegotiatorImpl
+  implements ColumnsSchemaNegotiator {
+
+private final ColumnsScanFramework framework;
+
+public ColumnsSchemaNegotiatorImpl(ColumnsScanFramework framework,
+ShimBatchReader shim) {
+  super(framework, shim);
 
 Review comment:
   Super already has framework, can we use it from super and not store it in 
`ColumnsSchemaNegotiatorImpl` class?


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row set-based scan framework

2019-01-24 Thread GitBox
arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row 
set-based scan framework
URL: https://github.com/apache/drill/pull/1618#discussion_r250615178
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileMetadataManager.java
 ##
 @@ -0,0 +1,239 @@
+/*
+ * 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.physical.impl.scan.file;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.directory.api.util.Strings;
+import org.apache.drill.common.map.CaseInsensitiveMap;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.physical.impl.scan.project.ColumnProjection;
+import org.apache.drill.exec.physical.impl.scan.project.ConstantColumnLoader;
+import org.apache.drill.exec.physical.impl.scan.project.MetadataManager;
+import org.apache.drill.exec.physical.impl.scan.project.ResolvedTuple;
+import 
org.apache.drill.exec.physical.impl.scan.project.ScanLevelProjection.ScanProjectionParser;
+import 
org.apache.drill.exec.physical.impl.scan.project.SchemaLevelProjection.SchemaProjectionResolver;
+import org.apache.drill.exec.physical.impl.scan.project.VectorSource;
+import org.apache.drill.exec.physical.rowSet.ResultVectorCache;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.server.options.OptionSet;
+import org.apache.drill.exec.store.ColumnExplorer.ImplicitFileColumns;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.hadoop.fs.Path;
+
+import 
org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTesting;
+
+public class FileMetadataManager implements MetadataManager, 
SchemaProjectionResolver, VectorSource {
+
+  // Input
+
+  private Path scanRootDir;
+  private FileMetadata currentFile;
+
+  // Config
+
+  protected final String partitionDesignator;
+  protected List implicitColDefns = new ArrayList<>();
+  protected Map fileMetadataColIndex = 
CaseInsensitiveMap.newHashMap();
+  protected boolean useLegacyWildcardExpansion = true;
+  private final FileMetadataColumnsParser parser;
+
+  // Internal state
+
+  private ResultVectorCache vectorCache;
+  private final List metadataColumns = new ArrayList<>();
+  private ConstantColumnLoader loader;
+  private VectorContainer outputContainer;
+  private final int partitionCount;
+
+  /**
+   * Specifies whether to plan based on the legacy meaning of "*". See
+   * https://issues.apache.org/jira/browse/DRILL-5542;>DRILL-5542.
+   * If true, then the star column includes implicit and partition
+   * columns. If false, then star matches only table columns.
+   *
+   * @param optionManager access to the options for this query; used
+   * too look up custom names for the metadata columns
+   * @param useLegacyWildcardExpansion true to use the legacy plan, false to 
use the revised
+   * semantics
+   * @param rootDir when scanning multiple files, the root directory for
+   * the file set. Unfortunately, the planner is ambiguous on this one; if the
+   * query is against a single file, then this variable holds the name of that
+   * one file, rather than a directory
+   * @param files the set of files to scan. Used to compute the maximum 
partition
+   * depth across all readers in this fragment
+   *
+   * @return this builder
+   */
+
+  public FileMetadataManager(OptionSet optionManager,
+  boolean useLegacyWildcardExpansion,
+  Path rootDir, List files) {
+this.useLegacyWildcardExpansion = useLegacyWildcardExpansion;
+scanRootDir = rootDir;
+
+partitionDesignator = 
optionManager.getString(ExecConstants.FILESYSTEM_PARTITION_COLUMN_LABEL);
+for (ImplicitFileColumns e : ImplicitFileColumns.values()) {
+  String colName = optionManager.getString(e.optionName());
+  if (! Strings.isEmpty(colName)) {
+FileMetadataColumnDefn defn = new FileMetadataColumnDefn(colName, e);
+implicitColDefns.add(defn);
+fileMetadataColIndex.put(defn.colName, defn);
+  }
+}
+parser = new FileMetadataColumnsParser(this);
+
+// The files and root dir are optional.
+
+   

[GitHub] arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row set-based scan framework

2019-01-24 Thread GitBox
arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row 
set-based scan framework
URL: https://github.com/apache/drill/pull/1618#discussion_r250612100
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileMetadataColumnsParser.java
 ##
 @@ -0,0 +1,166 @@
+/*
+ * 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.physical.impl.scan.file;
+
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.drill.exec.physical.impl.scan.project.ColumnProjection;
+import org.apache.drill.exec.physical.impl.scan.project.ScanLevelProjection;
+import 
org.apache.drill.exec.physical.impl.scan.project.ScanLevelProjection.ScanProjectionParser;
+import 
org.apache.drill.exec.physical.rowSet.project.RequestedTuple.RequestedColumn;
+
+public class FileMetadataColumnsParser implements ScanProjectionParser {
+
+  static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(FileMetadataColumnsParser.class);
+
+  // Internal
+
+  private final FileMetadataManager metadataManager;
+  private final Pattern partitionPattern;
+  private ScanLevelProjection builder;
+
+  // Output
+
+  boolean hasMetadata;
+
+  public FileMetadataColumnsParser(FileMetadataManager metadataManager) {
+this.metadataManager = metadataManager;
+partitionPattern = Pattern.compile(metadataManager.partitionDesignator + 
"(\\d+)", Pattern.CASE_INSENSITIVE);
+  }
+
+  @Override
+  public void bind(ScanLevelProjection builder) {
+this.builder = builder;
+  }
+
+  @Override
+  public boolean parse(RequestedColumn inCol) {
+Matcher m = partitionPattern.matcher(inCol.name());
+if (m.matches()) {
+  return buildPartitionColumn(m, inCol);
+}
+
+FileMetadataColumnDefn defn = 
metadataManager.fileMetadataColIndex.get(inCol.name());
+if (defn != null) {
+  return buildMetadataColumn(defn, inCol);
+}
+if (inCol.isWildcard()) {
+  buildWildcard();
+
+  // Don't consider this a match.
+}
+return false;
+  }
+
+  private boolean buildPartitionColumn(Matcher m, RequestedColumn inCol) {
+
+// If the projected column is a map or array, then it shadows the
+// partition column. Example: dir0.x, dir0[2].
+
+if (! inCol.isSimple()) {
+  logger.warn("Partition column {} is shadowed by a projected {}",
+  inCol.name(), inCol.summary());
+  return false;
+}
+if (builder.hasWildcard()) {
+  wildcardAndMetadataError();
+}
+
+// Partition column
+
+builder.addMetadataColumn(
+new PartitionColumn(
+  inCol.name(),
+  Integer.parseInt(m.group(1;
+hasMetadata = true;
+return true;
+  }
+
+  private boolean buildMetadataColumn(FileMetadataColumnDefn defn,
+  RequestedColumn inCol) {
+
+// If the projected column is a map or array, then it shadows the
+// metadata column. Example: filename.x, filename[2].
+
+if (! inCol.isSimple()) {
+  logger.warn("File metadata column {} is shadowed by a projected {}",
+  inCol.name(), inCol.summary());
+  return false;
+}
+if (builder.hasWildcard()) {
+  wildcardAndMetadataError();
+}
+
+// File metadata (implicit) column
+
+builder.addMetadataColumn(new FileMetadataColumn(inCol.name(), defn));
+hasMetadata = true;
+return true;
+  }
+
+  private void buildWildcard() {
+if (hasMetadata) {
+  wildcardAndMetadataError();
+}
+if (metadataManager.useLegacyWildcardExpansion) {
+
+  // Star column: this is a SELECT * query.
+
+  // Old-style wildcard handling inserts all metadata columns in
+  // the scanner, removes them in Project.
+  // Fill in the file metadata columns. Can do here because the
+  // set is constant across all files.
+
+  expandWildcard();
+  hasMetadata = true;
+}
+  }
+
+  protected void expandWildcard() {
+
+// Legacy wildcard expansion: include the file metadata and
+// file partitions for this file.
+// This is a disadvantage for a * query: files at different directory
+// levels will have different numbers of columns. Would be better to
+// return this data 

[GitHub] arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row set-based scan framework

2019-01-24 Thread GitBox
arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row 
set-based scan framework
URL: https://github.com/apache/drill/pull/1618#discussion_r250616495
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/AbstractScanFramework.java
 ##
 @@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.scan.framework;
+
+import java.util.List;
+
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.physical.impl.scan.ScanOperatorEvents;
+import org.apache.drill.exec.physical.impl.scan.project.ScanSchemaOrchestrator;
+
+/**
+ * Basic scan framework for a "managed" reader which uses the scan schema
+ * mechanisms encapsulated in the scan schema orchestrator. Handles binding
+ * scan events to the scan orchestrator so that the scan schema is evolved
+ * as the scan progresses. Subclasses are responsible for creating the actual
+ * reader, which requires a framework-specific schema negotiator to be passed
+ * to the reader.
+ * 
+ * This framework is a bridge between operator logic and the scan projection
+ * internals. It gathers scan-specific options, then sets
 
 Review comment:
   ` It gathers scan-specific options, then sets then on the scan orchestrator 
at the right time.`
   Looks like we have extra `then` here.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row set-based scan framework

2019-01-24 Thread GitBox
arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row 
set-based scan framework
URL: https://github.com/apache/drill/pull/1618#discussion_r250591376
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/ScanOperatorExec.java
 ##
 @@ -0,0 +1,278 @@
+/*
+ * 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.physical.impl.scan;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.physical.impl.protocol.BatchAccessor;
+import org.apache.drill.exec.physical.impl.protocol.OperatorExec;
+import org.apache.drill.exec.physical.impl.protocol.VectorContainerAccessor;
+
+import 
org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Implementation of the revised scan operator that uses a mutator aware of
+ * batch sizes. This is the successor to {@link ScanBatch} and should be used
+ * by all new scan implementations.
+ *
+ * Scanner Framework
+ *
+ * Acts as an adapter between the operator protocol and the row reader
+ * protocol.
+ * 
+ * The scan operator itself is simply a framework for handling a set of 
readers;
+ * it knows nothing other than the interfaces of the components it works with;
+ * delegating all knowledge of schemas, projection, reading and the like to
+ * implementations of those interfaces. Because that work is complex, a set
+ * of frameworks exist to handle most common use cases, but a specialized 
reader
+ * can create a framework or reader from scratch.
+ * 
+ * Error handling in this class is minimal: the enclosing record batch iterator
+ * is responsible for handling exceptions. Error handling relies on the fact
+ * that the iterator will call close() regardless of which exceptions
+ * are thrown.
+ *
+ * Protocol
+ *
+ * The scanner works directly with two other interfaces
+ * 
+ * The {@link ScanOperatorEvents} implementation provides the set of readers to
+ * use. This class can simply maintain a list, or can create the reader on
+ * demand.
+ * 
+ * More subtly, the factory also handles projection issues and manages vectors
+ * across subsequent readers. A number of factories are available for the most
+ * common cases. Extend these to implement a version specific to a data source.
+ * 
+ * The {@link RowBatchReader} is a surprisingly minimal interface that
+ * nonetheless captures the essence of reading a result set as a set of 
batches.
+ * The factory implementations mentioned above implement this interface to 
provide
+ * commonly-used services, the most important of which is access to a
+ * {#link ResultSetLoader} to write values into value vectors.
+ *
+ * Schema Versions
+ * Readers may change schemas from time to time. To track such changes,
+ * this implementation tracks a batch schema version, maintained by comparing
+ * one schema with the next.
+ * 
+ * Readers can discover columns as they read data, such as with any JSON-based
+ * format. In this case, the row set mutator also provides a schema version,
+ * but a fine-grained one that changes each time a column is added.
+ * 
+ * The two schema versions serve different purposes and are not 
interchangeable.
+ * For example, if a scan reads two files, both will build up their own 
schemas,
+ * each increasing its internal version number as work proceeds. But, at the
+ * end of each batch, the schemas may (and, in fact, should) be identical,
+ * which is the schema version downstream operators care about.
+ */
+
+public class ScanOperatorExec implements OperatorExec {
+
+  private enum State { START, READER, END, FAILED, CLOSED }
+
+  static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(ScanOperatorExec.class);
+
+  private final ScanOperatorEvents factory;
+  protected final VectorContainerAccessor containerAccessor = new 
VectorContainerAccessor();
+  private State state = State.START;
+  protected OperatorContext context;
+  private int readerCount;
+  private ReaderState readerState;
+
+  public ScanOperatorExec(ScanOperatorEvents factory) {
+this.factory = 

[GitHub] arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row set-based scan framework

2019-01-24 Thread GitBox
arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row 
set-based scan framework
URL: https://github.com/apache/drill/pull/1618#discussion_r250586846
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/ReaderState.java
 ##
 @@ -0,0 +1,287 @@
+/*
+ * 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.physical.impl.scan;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.record.VectorContainer;
+
+/**
+ * Manages a row batch reader through its lifecycle. Created when the reader
+ * is opened, discarded when the reader is closed. Encapsulates state that
+ * follows the life of the reader. This moves common scanner state out of
+ * each reader in order to make the reader as simple as possible.
+ * 
+ * This class is private to the scan operator and is not meant to be used,
+ * or even visible, outside of that operator itself. Instead, all 
reader-specific
+ * functionality should be in the {@link RowBatchReader} subclass.
+ */
+
+class ReaderState {
+  static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(ReaderState.class);
+
+  private enum State { START, LOOK_AHEAD, LOOK_AHEAD_WITH_EOF, ACTIVE, EOF, 
CLOSED };
+
+  final ScanOperatorExec scanOp;
+  private final RowBatchReader reader;
+  private State state = State.START;
+  private VectorContainer lookahead;
+  private int schemaVersion = -1;
+
+  public ReaderState(ScanOperatorExec scanOp, RowBatchReader reader) {
+this.scanOp = scanOp;
+this.reader = reader;
+  }
+
+  /**
+   * Open the next available reader, if any, preparing both the
+   * reader and table loader.
+   * @return true if another reader is active, false if no more
+   * readers are available
+   */
+
+  boolean open() {
+
+// Open the reader. This can fail. if it does, clean up.
+
+try {
+
+  // The reader can return a "soft" failure: the open worked, but
+  // the file is empty, non-existent or some other form of "no data."
+  // Handle this by immediately moving to EOF. The scanner will quietly
+  // pass over this reader and move onto the next, if any.
+
+  if (! reader.open()) {
+state = State.EOF;
+return false;
+  }
+
+// When catching errors, leave the reader member set;
+// we must close it on close() later.
+
+} catch (UserException e) {
+
+  // Throw user exceptions as-is
+
+  throw e;
+} catch (Throwable t) {
+
+  // Wrap all others in a user exception.
+
+  throw UserException.executionError(t)
+.addContext("Open failed for reader", reader.name())
+.build(logger);
+}
+
+state = State.ACTIVE;
+return true;
+  }
+
+  /**
+   * Prepare the schema for this reader. Called for the first reader within a
+   * scan batch, if the reader returns true from open(). If
+   * this is an early-schema reader, then the result set loader already has
+   * the proper value vectors set up. If this is a late-schema reader, we must
+   * read one batch to get the schema, then set aside the data for the next
+   * call to next().
+   * 
+   * Semantics for all readers:
+   * 
+   * If the file was not found, open() returned false and this
+   * method should never be called.
+   * 
+   * 
+   * Semantics for early-schema readers:
+   * 
+   * If if turned out that the file was
+   * empty when trying to read the schema, open() returned false
+   * and this method should never be called.
+   * Otherwise, if a schema was available, then the schema is already
+   * set up in the result set loader as the result of schema negotiation, and
+   * this method simply returns true.
+   * 
+   * 
+   * Semantics for late-schema readers:
+   * 
+   * This method will ask the reader to
+   * read a batch. If the reader hits EOF before finding any data, this method
+   * will return false, indicating that no schema is available.
+   * If the reader can read enough of the file to
+   * figure out the schema, but the file has no data, then this method will
+   * return true and a schema will be available. The first call to
+   * next() will report EOF.
+   * Otherwise, this method returns 

[GitHub] arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row set-based scan framework

2019-01-24 Thread GitBox
arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row 
set-based scan framework
URL: https://github.com/apache/drill/pull/1618#discussion_r250595479
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/ColumnsArrayParser.java
 ##
 @@ -0,0 +1,173 @@
+/*
+ * 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.physical.impl.scan.columns;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.physical.impl.scan.project.ColumnProjection;
+import org.apache.drill.exec.physical.impl.scan.project.ScanLevelProjection;
+import 
org.apache.drill.exec.physical.impl.scan.project.ScanLevelProjection.ScanProjectionParser;
+import org.apache.drill.exec.physical.impl.scan.project.UnresolvedColumn;
+import org.apache.drill.exec.physical.rowSet.project.RequestedColumnImpl;
+import 
org.apache.drill.exec.physical.rowSet.project.RequestedTuple.RequestedColumn;
+import org.apache.drill.exec.store.easy.text.compliant.RepeatedVarCharOutput;
+
+/**
+ * Parses the `columns` array. Doing so is surprisingly complex.
+ * 
+ * Depending on what is known about the input file, the `columns`
+ * array may be required or optional.
+ * If the columns array is required, then the wildcard (`*`)
+ * expands to `columns`.
+ * If the columns array appears, then no other table columns
+ * can appear.
+ * If the columns array appears, then the wildcard cannot also
+ * appear, unless that wildcard expanded to be `columns` as
+ * described above.
+ * The query can select specific elements such as `columns`[2].
+ * In this case, only array elements can appear, not the unindexed
+ * `columns` column.
+ * 
+ * 
+ * It falls to this parser to detect a not-uncommon user error, a
+ * query such as the following:
+ * SELECT max(columns[1]) AS col1
+ * FROM cp.`textinput/input1.csv`
+ * WHERE col1 IS NOT NULL
+ * 
+ * In standard SQL, column aliases are not allowed in the WHERE
+ * clause. So, Drill will push two columns down to the scan operator:
+ * `columns`[1] and `col1`. This parser will detect the "extra"
+ * columns and must provide a message that helps the user identify
+ * the likely original problem.
+ */
+
+public class ColumnsArrayParser implements ScanProjectionParser {
+  static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(ColumnsArrayParser.class);
+
+  // Config
+
+  private final boolean requireColumnsArray;
+
+  // Internals
+
+  private ScanLevelProjection builder;
+
+  // Output
+
+  private UnresolvedColumnsArrayColumn columnsArrayCol;
+
+  public ColumnsArrayParser(boolean requireColumnsArray) {
+this.requireColumnsArray = requireColumnsArray;
+  }
+
+  @Override
+  public void bind(ScanLevelProjection builder) {
+this.builder = builder;
+  }
+
+  @Override
+  public boolean parse(RequestedColumn inCol) {
+if (requireColumnsArray && inCol.isWildcard()) {
+  expandWildcard();
+  return true;
+}
+if (! inCol.nameEquals(ColumnsArrayManager.COLUMNS_COL)) {
+  return false;
+}
+
+// The columns column cannot be a map. That is, the following is
+// not allowed: columns.foo.
+
+if (inCol.isTuple()) {
+  throw UserException
+.validationError()
+.message("{} has mep elements, but cannot be a map", inCol.name())
+.build(logger);
+}
+
+if (inCol.isArray()) {
+  int maxIndex = inCol.maxIndex();
+  if (maxIndex > RepeatedVarCharOutput.MAXIMUM_NUMBER_COLUMNS) {
 
 Review comment:
   What user is expected to do when he hits such limitation?


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row set-based scan framework

2019-01-24 Thread GitBox
arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row 
set-based scan framework
URL: https://github.com/apache/drill/pull/1618#discussion_r250613878
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileMetadataManager.java
 ##
 @@ -0,0 +1,239 @@
+/*
+ * 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.physical.impl.scan.file;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.directory.api.util.Strings;
+import org.apache.drill.common.map.CaseInsensitiveMap;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.physical.impl.scan.project.ColumnProjection;
+import org.apache.drill.exec.physical.impl.scan.project.ConstantColumnLoader;
+import org.apache.drill.exec.physical.impl.scan.project.MetadataManager;
+import org.apache.drill.exec.physical.impl.scan.project.ResolvedTuple;
+import 
org.apache.drill.exec.physical.impl.scan.project.ScanLevelProjection.ScanProjectionParser;
+import 
org.apache.drill.exec.physical.impl.scan.project.SchemaLevelProjection.SchemaProjectionResolver;
+import org.apache.drill.exec.physical.impl.scan.project.VectorSource;
+import org.apache.drill.exec.physical.rowSet.ResultVectorCache;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.server.options.OptionSet;
+import org.apache.drill.exec.store.ColumnExplorer.ImplicitFileColumns;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.hadoop.fs.Path;
+
+import 
org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTesting;
+
+public class FileMetadataManager implements MetadataManager, 
SchemaProjectionResolver, VectorSource {
+
+  // Input
+
+  private Path scanRootDir;
+  private FileMetadata currentFile;
+
+  // Config
+
+  protected final String partitionDesignator;
+  protected List implicitColDefns = new ArrayList<>();
+  protected Map fileMetadataColIndex = 
CaseInsensitiveMap.newHashMap();
+  protected boolean useLegacyWildcardExpansion = true;
 
 Review comment:
   `useLegacyWildcardExpansion` is passed in the constructor, no need to define 
it explicitly.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row set-based scan framework

2019-01-24 Thread GitBox
arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row 
set-based scan framework
URL: https://github.com/apache/drill/pull/1618#discussion_r250585447
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/ReaderState.java
 ##
 @@ -0,0 +1,287 @@
+/*
+ * 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.physical.impl.scan;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.record.VectorContainer;
+
+/**
+ * Manages a row batch reader through its lifecycle. Created when the reader
+ * is opened, discarded when the reader is closed. Encapsulates state that
+ * follows the life of the reader. This moves common scanner state out of
+ * each reader in order to make the reader as simple as possible.
+ * 
+ * This class is private to the scan operator and is not meant to be used,
+ * or even visible, outside of that operator itself. Instead, all 
reader-specific
+ * functionality should be in the {@link RowBatchReader} subclass.
+ */
+
+class ReaderState {
+  static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(ReaderState.class);
+
+  private enum State { START, LOOK_AHEAD, LOOK_AHEAD_WITH_EOF, ACTIVE, EOF, 
CLOSED };
+
+  final ScanOperatorExec scanOp;
+  private final RowBatchReader reader;
+  private State state = State.START;
+  private VectorContainer lookahead;
+  private int schemaVersion = -1;
+
+  public ReaderState(ScanOperatorExec scanOp, RowBatchReader reader) {
+this.scanOp = scanOp;
+this.reader = reader;
+  }
+
+  /**
+   * Open the next available reader, if any, preparing both the
+   * reader and table loader.
+   * @return true if another reader is active, false if no more
+   * readers are available
+   */
+
+  boolean open() {
+
+// Open the reader. This can fail. if it does, clean up.
+
+try {
+
+  // The reader can return a "soft" failure: the open worked, but
+  // the file is empty, non-existent or some other form of "no data."
+  // Handle this by immediately moving to EOF. The scanner will quietly
+  // pass over this reader and move onto the next, if any.
+
+  if (! reader.open()) {
+state = State.EOF;
+return false;
+  }
+
+// When catching errors, leave the reader member set;
+// we must close it on close() later.
+
+} catch (UserException e) {
+
+  // Throw user exceptions as-is
+
+  throw e;
+} catch (Throwable t) {
+
+  // Wrap all others in a user exception.
+
+  throw UserException.executionError(t)
+.addContext("Open failed for reader", reader.name())
+.build(logger);
+}
+
+state = State.ACTIVE;
+return true;
+  }
+
+  /**
+   * Prepare the schema for this reader. Called for the first reader within a
+   * scan batch, if the reader returns true from open(). If
+   * this is an early-schema reader, then the result set loader already has
+   * the proper value vectors set up. If this is a late-schema reader, we must
+   * read one batch to get the schema, then set aside the data for the next
+   * call to next().
+   * 
+   * Semantics for all readers:
+   * 
+   * If the file was not found, open() returned false and this
+   * method should never be called.
+   * 
+   * 
+   * Semantics for early-schema readers:
+   * 
+   * If if turned out that the file was
+   * empty when trying to read the schema, open() returned false
+   * and this method should never be called.
+   * Otherwise, if a schema was available, then the schema is already
+   * set up in the result set loader as the result of schema negotiation, and
+   * this method simply returns true.
+   * 
+   * 
+   * Semantics for late-schema readers:
+   * 
+   * This method will ask the reader to
+   * read a batch. If the reader hits EOF before finding any data, this method
+   * will return false, indicating that no schema is available.
+   * If the reader can read enough of the file to
+   * figure out the schema, but the file has no data, then this method will
+   * return true and a schema will be available. The first call to
+   * next() will report EOF.
+   * Otherwise, this method returns 

[GitHub] arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row set-based scan framework

2019-01-24 Thread GitBox
arina-ielchiieva commented on a change in pull request #1618: DRILL-6950: Row 
set-based scan framework
URL: https://github.com/apache/drill/pull/1618#discussion_r250575132
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/ReaderState.java
 ##
 @@ -0,0 +1,287 @@
+/*
+ * 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.physical.impl.scan;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.record.VectorContainer;
+
+/**
+ * Manages a row batch reader through its lifecycle. Created when the reader
+ * is opened, discarded when the reader is closed. Encapsulates state that
+ * follows the life of the reader. This moves common scanner state out of
+ * each reader in order to make the reader as simple as possible.
+ * 
+ * This class is private to the scan operator and is not meant to be used,
+ * or even visible, outside of that operator itself. Instead, all 
reader-specific
+ * functionality should be in the {@link RowBatchReader} subclass.
+ */
+
+class ReaderState {
+  static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(ReaderState.class);
+
+  private enum State { START, LOOK_AHEAD, LOOK_AHEAD_WITH_EOF, ACTIVE, EOF, 
CLOSED };
 
 Review comment:
   Could you please add java-doc for each state, describing its purpose?


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services