You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by vi...@apache.org on 2019/03/05 15:38:27 UTC

[drill] branch master updated (2c3e2de -> 3d29faf)

This is an automated email from the ASF dual-hosted git repository.

vitalii pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git.


    from 2c3e2de  DRILL-6642: Update protocol-buffers version
     new 7e3b459  DRILL-7074: Scan framework fixes and enhancements
     new 3d29faf  DRILL-5603: Replace String file paths to Hadoop Path  - replaced all String path representation with org.apache.hadoop.fs.Path  - added PathSerDe.Se JSON serializer  - refactoring of DFSPartitionLocation code by leveraging existing listPartitionValues() functionality

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../exec/store/mapr/db/MapRDBFormatPlugin.java     |   4 +-
 .../store/mapr/streams/StreamsFormatPlugin.java    |  12 +-
 .../exec/store/syslog/SyslogRecordReader.java      |   3 +-
 .../exec/planner/sql/HivePartitionDescriptor.java  |   9 +-
 .../exec/planner/sql/HivePartitionLocation.java    |   8 +-
 .../hive/HiveDrillNativeParquetRowGroupScan.java   |   2 +-
 .../store/hive/HiveDrillNativeParquetScan.java     |   6 +-
 .../HiveDrillNativeParquetScanBatchCreator.java    |   5 +-
 .../drill/exec/store/hive/HivePartitionHolder.java |  11 +-
 .../apache/drill/exec/physical/PhysicalPlan.java   |   3 +-
 .../exec/physical/base/AbstractGroupScan.java      |   3 +-
 .../apache/drill/exec/physical/base/GroupScan.java |   5 +-
 .../drill/exec/physical/base/SchemalessScan.java   |   9 +-
 .../drill/exec/physical/impl/scan/ReaderState.java |  26 +-
 .../exec/physical/impl/scan/RowBatchReader.java    |  20 +-
 .../physical/impl/scan/ScanOperatorEvents.java     |  42 +-
 .../impl/scan/columns/ColumnsArrayParser.java      |  52 +-
 .../impl/scan/file/BaseFileScanFramework.java      |   2 +-
 .../exec/physical/impl/scan/file/FileMetadata.java |   2 +-
 .../impl/scan/file/FileMetadataColumnsParser.java  |  71 +-
 .../impl/scan/file/FileMetadataManager.java        |  34 +
 .../physical/impl/scan/file/FileScanFramework.java |  37 +-
 .../physical/impl/scan/file/MetadataColumn.java    |  13 +
 .../impl/scan/framework/AbstractScanFramework.java |  76 ++
 .../impl/scan/framework/SchemaNegotiator.java      |   6 +-
 .../impl/scan/framework/SchemaNegotiatorImpl.java  |   6 +-
 .../impl/scan/framework/ShimBatchReader.java       |  15 +-
 .../physical/impl/scan/framework/package-info.java |  26 +-
 .../exec/physical/impl/scan/package-info.java      |  85 +-
 .../scan/project/ExplicitSchemaProjection.java     |   1 +
 .../scan/project/ReaderSchemaOrchestrator.java     | 230 +++++
 .../impl/scan/project/ScanLevelProjection.java     |  66 +-
 .../impl/scan/project/ScanSchemaOrchestrator.java  | 241 +-----
 .../impl/scan/project/SchemaLevelProjection.java   |  11 +-
 .../physical/impl/scan/project/package-info.java   |  50 ++
 .../rowSet/project/ImpliedTupleRequest.java        |   2 +-
 .../rowSet/project/RequestedTupleImpl.java         |  44 +-
 .../exec/planner/AbstractPartitionDescriptor.java  |   3 +-
 .../exec/planner/DFSDirPartitionLocation.java      |  18 +-
 .../exec/planner/DFSFilePartitionLocation.java     |  25 +-
 .../planner/FileSystemPartitionDescriptor.java     | 102 +--
 .../exec/planner/ParquetPartitionDescriptor.java   |  18 +-
 .../exec/planner/ParquetPartitionLocation.java     |   8 +-
 .../drill/exec/planner/PartitionDescriptor.java    |  44 +-
 .../drill/exec/planner/PartitionLocation.java      |  14 +-
 .../drill/exec/planner/PhysicalPlanReader.java     |  53 +-
 .../exec/planner/SimplePartitionLocation.java      |   5 +-
 .../planner/logical/partition/PruneScanRule.java   |  17 +-
 .../planner/sql/handlers/AnalyzeTableHandler.java  |  28 +-
 .../sql/handlers/RefreshMetadataHandler.java       |   4 +-
 .../{InstanceSerializer.java => PathSerDe.java}    |  20 +-
 .../apache/drill/exec/store/ColumnExplorer.java    |  46 +-
 .../drill/exec/store/avro/AvroDrillTable.java      |   4 +-
 .../drill/exec/store/avro/AvroRecordReader.java    |  12 +-
 .../apache/drill/exec/store/dfs/FileSelection.java | 161 ++--
 .../drill/exec/store/dfs/FormatSelection.java      |   8 +-
 .../drill/exec/store/dfs/MetadataContext.java      |   7 +-
 .../drill/exec/store/dfs/ReadEntryFromHDFS.java    |   3 +-
 .../drill/exec/store/dfs/ReadEntryWithPath.java    |  13 +-
 .../exec/store/dfs/easy/EasyFormatPlugin.java      |   2 +-
 .../drill/exec/store/dfs/easy/EasyGroupScan.java   |  33 +-
 .../drill/exec/store/dfs/easy/EasySubScan.java     |  22 +-
 .../apache/drill/exec/store/dfs/easy/FileWork.java |   8 +-
 .../exec/store/direct/MetadataDirectGroupScan.java |  10 +-
 .../exec/store/easy/json/JSONRecordReader.java     |  13 +-
 .../sequencefile/SequenceFileFormatPlugin.java     |   2 +-
 .../exec/store/easy/text/TextFormatPlugin.java     |   2 +-
 .../exec/store/httpd/HttpdLogFormatPlugin.java     |   2 +-
 .../drill/exec/store/image/ImageFormatPlugin.java  |   9 +-
 .../drill/exec/store/image/ImageRecordReader.java  |   4 +-
 .../drill/exec/store/log/LogRecordReader.java      |   5 +-
 .../store/parquet/AbstractParquetGroupScan.java    |  17 +-
 .../parquet/AbstractParquetScanBatchCreator.java   |  11 +-
 .../drill/exec/store/parquet/ParquetGroupScan.java |  42 +-
 .../store/parquet/ParquetGroupScanStatistics.java  |   5 +-
 .../exec/store/parquet/ParquetRowGroupScan.java    |  11 +-
 .../store/parquet/ParquetScanBatchCreator.java     |   3 +-
 .../drill/exec/store/parquet/RowGroupInfo.java     |   3 +-
 .../exec/store/parquet/RowGroupReadEntry.java      |   3 +-
 .../parquet/columnreaders/ParquetRecordReader.java |  24 +-
 .../exec/store/parquet/metadata/Metadata.java      |  37 +-
 .../exec/store/parquet/metadata/MetadataBase.java  |   5 +-
 .../store/parquet/metadata/MetadataPathUtils.java  |  29 +-
 .../exec/store/parquet/metadata/Metadata_V1.java   |  13 +-
 .../exec/store/parquet/metadata/Metadata_V2.java   |  15 +-
 .../exec/store/parquet/metadata/Metadata_V3.java   |  15 +-
 .../parquet/metadata/ParquetTableMetadataDirs.java |   7 +-
 .../exec/store/parquet2/DrillParquetReader.java    |   2 +-
 .../drill/exec/store/pcap/PcapRecordReader.java    |   4 +-
 .../exec/store/pcapng/PcapngRecordReader.java      |   4 +-
 .../drill/exec/store/schedule/BlockMapBuilder.java |  22 +-
 .../exec/store/schedule/CompleteFileWork.java      |  42 +-
 .../drill/exec/util/DrillFileSystemUtil.java       |  11 +
 .../test/java/org/apache/drill/PlanTestBase.java   |   3 +-
 .../apache/drill/exec/TestPathSerialization.java   |  65 ++
 .../exec/physical/impl/scan/TestColumnsArray.java  | 172 +++-
 .../impl/scan/TestColumnsArrayFramework.java       |  19 +-
 .../physical/impl/scan/TestColumnsArrayParser.java |  46 +-
 .../impl/scan/TestConstantColumnLoader.java        | 153 ----
 .../impl/scan/TestFileMetadataColumnParser.java    | 170 +++-
 .../impl/scan/TestFileMetadataProjection.java      |   4 +-
 .../physical/impl/scan/TestFileScanFramework.java  |  13 +-
 .../physical/impl/scan/TestNullColumnLoader.java   | 281 ------
 .../physical/impl/scan/TestRowBatchMerger.java     | 459 ----------
 .../physical/impl/scan/TestScanBatchWriters.java   |   3 +
 .../impl/scan/TestScanLevelProjection.java         | 223 -----
 .../physical/impl/scan/TestScanOperatorExec.java   |  11 +-
 .../impl/scan/TestScanOrchestratorEarlySchema.java | 255 ++----
 .../impl/scan/TestScanOrchestratorLateSchema.java  |   6 +-
 .../impl/scan/TestScanOrchestratorMetadata.java    |  77 +-
 .../impl/scan/TestSchemaLevelProjection.java       | 557 ------------
 .../physical/impl/scan/TestSchemaSmoothing.java    | 946 ---------------------
 .../scan/project/TestConstantColumnLoader.java     |  45 +
 .../impl/scan/project/TestScanLevelProjection.java | 113 ++-
 .../impl/scan/project/TestSchemaSmoothing.java     | 250 +++++-
 .../exec/physical/unit/MiniPlanUnitTestBase.java   |  13 +-
 .../drill/exec/physical/unit/TestMiniPlan.java     |   5 +-
 .../exec/physical/unit/TestNullInputMiniPlan.java  |   8 +-
 .../drill/exec/store/CachedSingleFileSystem.java   |   8 +-
 .../exec/store/dfs/TestDFSPartitionLocation.java   |  54 ++
 .../drill/exec/store/dfs/TestFileSelection.java    |   7 +-
 .../store/parquet/ParquetRecordReaderTest.java     |   4 +-
 .../drill/exec/store/store/TestAssignment.java     |   3 +-
 .../apache/drill/test/PhysicalOpUnitTestBase.java  |   5 +-
 .../java/org/apache/drill/test/QueryBuilder.java   |   2 +-
 125 files changed, 2469 insertions(+), 3799 deletions(-)
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ReaderSchemaOrchestrator.java
 copy exec/java-exec/src/main/java/org/apache/drill/exec/serialization/{InstanceSerializer.java => PathSerDe.java} (60%)
 create mode 100644 exec/java-exec/src/test/java/org/apache/drill/exec/TestPathSerialization.java
 delete mode 100644 exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestConstantColumnLoader.java
 delete mode 100644 exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestNullColumnLoader.java
 delete mode 100644 exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestRowBatchMerger.java
 delete mode 100644 exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanLevelProjection.java
 delete mode 100644 exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestSchemaLevelProjection.java
 delete mode 100644 exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestSchemaSmoothing.java
 create mode 100644 exec/java-exec/src/test/java/org/apache/drill/exec/store/dfs/TestDFSPartitionLocation.java


[drill] 02/02: DRILL-5603: Replace String file paths to Hadoop Path - replaced all String path representation with org.apache.hadoop.fs.Path - added PathSerDe.Se JSON serializer - refactoring of DFSPartitionLocation code by leveraging existing listPartitionValues() functionality

Posted by vi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

vitalii pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git

commit 3d29faf81da593035f6bd38dd56d48e719afe7d4
Author: Vitalii Diravka <vi...@gmail.com>
AuthorDate: Mon Feb 18 22:30:36 2019 +0200

    DRILL-5603: Replace String file paths to Hadoop Path
     - replaced all String path representation with org.apache.hadoop.fs.Path
     - added PathSerDe.Se JSON serializer
     - refactoring of DFSPartitionLocation code by leveraging existing listPartitionValues() functionality
    
    closes #1657
---
 .../exec/store/mapr/db/MapRDBFormatPlugin.java     |   4 +-
 .../store/mapr/streams/StreamsFormatPlugin.java    |  12 +-
 .../exec/store/syslog/SyslogRecordReader.java      |   3 +-
 .../exec/planner/sql/HivePartitionDescriptor.java  |   9 +-
 .../exec/planner/sql/HivePartitionLocation.java    |   8 +-
 .../hive/HiveDrillNativeParquetRowGroupScan.java   |   2 +-
 .../store/hive/HiveDrillNativeParquetScan.java     |   6 +-
 .../HiveDrillNativeParquetScanBatchCreator.java    |   5 +-
 .../drill/exec/store/hive/HivePartitionHolder.java |  11 +-
 .../apache/drill/exec/physical/PhysicalPlan.java   |   3 +-
 .../exec/physical/base/AbstractGroupScan.java      |   3 +-
 .../apache/drill/exec/physical/base/GroupScan.java |   5 +-
 .../drill/exec/physical/base/SchemalessScan.java   |   9 +-
 .../impl/scan/file/BaseFileScanFramework.java      |   2 +-
 .../exec/physical/impl/scan/file/FileMetadata.java |   2 +-
 .../exec/planner/AbstractPartitionDescriptor.java  |   3 +-
 .../exec/planner/DFSDirPartitionLocation.java      |  18 ++-
 .../exec/planner/DFSFilePartitionLocation.java     |  25 +---
 .../planner/FileSystemPartitionDescriptor.java     | 102 ++++++-------
 .../exec/planner/ParquetPartitionDescriptor.java   |  18 +--
 .../exec/planner/ParquetPartitionLocation.java     |   8 +-
 .../drill/exec/planner/PartitionDescriptor.java    |  44 ++++--
 .../drill/exec/planner/PartitionLocation.java      |  14 +-
 .../drill/exec/planner/PhysicalPlanReader.java     |  53 +++----
 .../exec/planner/SimplePartitionLocation.java      |   5 +-
 .../planner/logical/partition/PruneScanRule.java   |  17 ++-
 .../planner/sql/handlers/AnalyzeTableHandler.java  |  28 ++--
 .../sql/handlers/RefreshMetadataHandler.java       |   4 +-
 .../PathSerDe.java}                                |  33 ++---
 .../apache/drill/exec/store/ColumnExplorer.java    |  46 +++---
 .../drill/exec/store/avro/AvroDrillTable.java      |   4 +-
 .../drill/exec/store/avro/AvroRecordReader.java    |  12 +-
 .../apache/drill/exec/store/dfs/FileSelection.java | 161 ++++++++++-----------
 .../drill/exec/store/dfs/FormatSelection.java      |   8 +-
 .../drill/exec/store/dfs/MetadataContext.java      |   7 +-
 .../drill/exec/store/dfs/ReadEntryFromHDFS.java    |   3 +-
 .../drill/exec/store/dfs/ReadEntryWithPath.java    |  13 +-
 .../exec/store/dfs/easy/EasyFormatPlugin.java      |   2 +-
 .../drill/exec/store/dfs/easy/EasyGroupScan.java   |  33 ++---
 .../drill/exec/store/dfs/easy/EasySubScan.java     |  22 +--
 .../apache/drill/exec/store/dfs/easy/FileWork.java |   8 +-
 .../exec/store/direct/MetadataDirectGroupScan.java |  10 +-
 .../exec/store/easy/json/JSONRecordReader.java     |  13 +-
 .../sequencefile/SequenceFileFormatPlugin.java     |   2 +-
 .../exec/store/easy/text/TextFormatPlugin.java     |   2 +-
 .../exec/store/httpd/HttpdLogFormatPlugin.java     |   2 +-
 .../drill/exec/store/image/ImageFormatPlugin.java  |   9 +-
 .../drill/exec/store/image/ImageRecordReader.java  |   4 +-
 .../drill/exec/store/log/LogRecordReader.java      |   5 +-
 .../store/parquet/AbstractParquetGroupScan.java    |  17 ++-
 .../parquet/AbstractParquetScanBatchCreator.java   |  11 +-
 .../drill/exec/store/parquet/ParquetGroupScan.java |  42 +++---
 .../store/parquet/ParquetGroupScanStatistics.java  |   5 +-
 .../exec/store/parquet/ParquetRowGroupScan.java    |  11 +-
 .../store/parquet/ParquetScanBatchCreator.java     |   3 +-
 .../drill/exec/store/parquet/RowGroupInfo.java     |   3 +-
 .../exec/store/parquet/RowGroupReadEntry.java      |   3 +-
 .../parquet/columnreaders/ParquetRecordReader.java |  24 ++-
 .../exec/store/parquet/metadata/Metadata.java      |  37 +++--
 .../exec/store/parquet/metadata/MetadataBase.java  |   5 +-
 .../store/parquet/metadata/MetadataPathUtils.java  |  29 ++--
 .../exec/store/parquet/metadata/Metadata_V1.java   |  13 +-
 .../exec/store/parquet/metadata/Metadata_V2.java   |  15 +-
 .../exec/store/parquet/metadata/Metadata_V3.java   |  15 +-
 .../parquet/metadata/ParquetTableMetadataDirs.java |   7 +-
 .../exec/store/parquet2/DrillParquetReader.java    |   2 +-
 .../drill/exec/store/pcap/PcapRecordReader.java    |   4 +-
 .../exec/store/pcapng/PcapngRecordReader.java      |   4 +-
 .../drill/exec/store/schedule/BlockMapBuilder.java |  22 +--
 .../exec/store/schedule/CompleteFileWork.java      |  42 ++++--
 .../drill/exec/util/DrillFileSystemUtil.java       |  11 ++
 .../test/java/org/apache/drill/PlanTestBase.java   |   3 +-
 .../apache/drill/exec/TestPathSerialization.java   |  65 +++++++++
 .../physical/impl/scan/TestFileScanFramework.java  |   2 +-
 .../exec/physical/unit/MiniPlanUnitTestBase.java   |  13 +-
 .../drill/exec/physical/unit/TestMiniPlan.java     |   5 +-
 .../exec/physical/unit/TestNullInputMiniPlan.java  |   8 +-
 .../drill/exec/store/CachedSingleFileSystem.java   |   8 +-
 .../exec/store/dfs/TestDFSPartitionLocation.java   |  54 +++++++
 .../drill/exec/store/dfs/TestFileSelection.java    |   7 +-
 .../store/parquet/ParquetRecordReaderTest.java     |   4 +-
 .../drill/exec/store/store/TestAssignment.java     |   3 +-
 .../apache/drill/test/PhysicalOpUnitTestBase.java  |   5 +-
 .../java/org/apache/drill/test/QueryBuilder.java   |   2 +-
 84 files changed, 741 insertions(+), 585 deletions(-)

diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatPlugin.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatPlugin.java
index 3011f4e..6501f8c 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatPlugin.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatPlugin.java
@@ -180,9 +180,9 @@ public class MapRDBFormatPlugin extends TableFormatPlugin {
    */
   @JsonIgnore
   public String getTableName(FileSelection selection) {
-    List<String> files = selection.getFiles();
+    List<Path> files = selection.getFiles();
     assert (files.size() == 1);
-    return files.get(0);
+    return files.get(0).toUri().getPath();
   }
 
 }
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/streams/StreamsFormatPlugin.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/streams/StreamsFormatPlugin.java
index 92f134f..2ddf752 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/streams/StreamsFormatPlugin.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/streams/StreamsFormatPlugin.java
@@ -36,7 +36,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
 public class StreamsFormatPlugin extends TableFormatPlugin {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StreamsFormatPlugin.class);
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StreamsFormatPlugin.class);
   private StreamsFormatMatcher matcher;
 
   public StreamsFormatPlugin(String name, DrillbitContext context, Configuration fsConf,
@@ -72,9 +72,8 @@ public class StreamsFormatPlugin extends TableFormatPlugin {
   }
 
   @Override
-  public AbstractGroupScan getGroupScan(String userName, FileSelection selection,
-      List<SchemaPath> columns) throws IOException {
-    List<String> files = selection.getFiles();
+  public AbstractGroupScan getGroupScan(String userName, FileSelection selection, List<SchemaPath> columns) {
+    List<Path> files = selection.getFiles();
     assert (files.size() == 1);
     //TableProperties props = getMaprFS().getTableProperties(new Path(files.get(0)));
     throw UserException.unsupportedError().message("MapR streams can not be querried at this time.").build(logger);
@@ -86,13 +85,12 @@ public class StreamsFormatPlugin extends TableFormatPlugin {
   }
 
   @Override
-  public DrillStatsTable.TableStatistics readStatistics(FileSystem fs, Path statsTablePath) throws IOException {
+  public DrillStatsTable.TableStatistics readStatistics(FileSystem fs, Path statsTablePath) {
     throw new UnsupportedOperationException("unimplemented");
   }
 
   @Override
-  public void writeStatistics(DrillStatsTable.TableStatistics statistics, FileSystem fs, Path statsTablePath) throws IOException {
+  public void writeStatistics(DrillStatsTable.TableStatistics statistics, FileSystem fs, Path statsTablePath) {
     throw new UnsupportedOperationException("unimplemented");
   }
-
 }
diff --git a/contrib/format-syslog/src/main/java/org/apache/drill/exec/store/syslog/SyslogRecordReader.java b/contrib/format-syslog/src/main/java/org/apache/drill/exec/store/syslog/SyslogRecordReader.java
index a198e34..4b2831c 100644
--- a/contrib/format-syslog/src/main/java/org/apache/drill/exec/store/syslog/SyslogRecordReader.java
+++ b/contrib/format-syslog/src/main/java/org/apache/drill/exec/store/syslog/SyslogRecordReader.java
@@ -32,7 +32,6 @@ import org.apache.drill.exec.store.dfs.DrillFileSystem;
 import org.apache.drill.exec.store.dfs.easy.FileWork;
 import org.apache.drill.exec.vector.complex.impl.VectorContainerWriter;
 import org.apache.drill.exec.vector.complex.writer.BaseWriter;
-import org.apache.hadoop.fs.Path;
 import org.realityforge.jsyslog.message.StructuredDataParameter;
 import org.realityforge.jsyslog.message.SyslogMessage;
 
@@ -95,7 +94,7 @@ public class SyslogRecordReader extends AbstractRecordReader {
   private void openFile() {
     InputStream in;
     try {
-      in = fileSystem.open(new Path(fileWork.getPath()));
+      in = fileSystem.open(fileWork.getPath());
     } catch (Exception e) {
       throw UserException
               .dataReadError(e)
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/HivePartitionDescriptor.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/HivePartitionDescriptor.java
index 25a0c08..a52b48d 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/HivePartitionDescriptor.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/HivePartitionDescriptor.java
@@ -36,6 +36,7 @@ import org.apache.drill.exec.store.hive.HiveUtilities;
 import org.apache.drill.exec.store.hive.HiveReadEntry;
 import org.apache.drill.exec.store.hive.HiveScan;
 import org.apache.drill.exec.vector.ValueVector;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.metastore.api.Partition;
 
 import java.util.BitSet;
@@ -87,9 +88,9 @@ public class HivePartitionDescriptor extends AbstractPartitionDescriptor {
   }
 
   @Override
-  public String getBaseTableLocation() {
+  public Path getBaseTableLocation() {
     HiveReadEntry origEntry = ((HiveScan) scanRel.getGroupScan()).getHiveReadEntry();
-    return origEntry.table.getTable().getSd().getLocation();
+    return new Path(origEntry.table.getTable().getSd().getLocation());
   }
 
   @Override
@@ -145,7 +146,7 @@ public class HivePartitionDescriptor extends AbstractPartitionDescriptor {
     List<PartitionLocation> locations = new LinkedList<>();
     HiveReadEntry origEntry = ((HiveScan) scanRel.getGroupScan()).getHiveReadEntry();
     for (Partition partition: origEntry.getPartitions()) {
-      locations.add(new HivePartitionLocation(partition.getValues(), partition.getSd().getLocation()));
+      locations.add(new HivePartitionLocation(partition.getValues(), new Path(partition.getSd().getLocation())));
     }
     locationSuperList = Lists.partition(locations, PartitionDescriptor.PARTITION_BATCH_SIZE);
     sublistsCreated = true;
@@ -170,7 +171,7 @@ public class HivePartitionDescriptor extends AbstractPartitionDescriptor {
     List<HiveTableWrapper.HivePartitionWrapper> newPartitions = Lists.newLinkedList();
 
     for (HiveTableWrapper.HivePartitionWrapper part: oldPartitions) {
-      String partitionLocation = part.getPartition().getSd().getLocation();
+      Path partitionLocation = new Path(part.getPartition().getSd().getLocation());
       for (PartitionLocation newPartitionLocation: newPartitionLocations) {
         if (partitionLocation.equals(newPartitionLocation.getEntirePartitionLocation())) {
           newPartitions.add(part);
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/HivePartitionLocation.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/HivePartitionLocation.java
index bb0efe8..25821f5 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/HivePartitionLocation.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/HivePartitionLocation.java
@@ -19,17 +19,19 @@ package org.apache.drill.exec.planner.sql;
 
 import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
 import org.apache.drill.exec.planner.SimplePartitionLocation;
+import org.apache.hadoop.fs.Path;
 
 import java.util.List;
 
 public class HivePartitionLocation extends SimplePartitionLocation {
-  private final String partitionLocation;
+  private final Path partitionLocation;
   private final List<String> partitionValues;
 
-  public HivePartitionLocation(final List<String> partitionValues, final String partitionLocation) {
+  public HivePartitionLocation(List<String> partitionValues, Path partitionLocation) {
     this.partitionValues = ImmutableList.copyOf(partitionValues);
     this.partitionLocation = partitionLocation;
   }
+
   @Override
   public String getPartitionValue(int index) {
     assert index < partitionValues.size();
@@ -37,7 +39,7 @@ public class HivePartitionLocation extends SimplePartitionLocation {
   }
 
   @Override
-  public String getEntirePartitionLocation() {
+  public Path getEntirePartitionLocation() {
     return partitionLocation;
   }
 }
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetRowGroupScan.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetRowGroupScan.java
index bea06e0..78e107a 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetRowGroupScan.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetRowGroupScan.java
@@ -124,7 +124,7 @@ public class HiveDrillNativeParquetRowGroupScan extends AbstractParquetRowGroupS
 
   @Override
   public Configuration getFsConf(RowGroupReadEntry rowGroupReadEntry) throws IOException {
-    Path path = new Path(rowGroupReadEntry.getPath()).getParent();
+    Path path = rowGroupReadEntry.getPath().getParent();
     return new ProjectionPusher().pushProjectionsAndFilters(
         new JobConf(HiveUtilities.generateHiveConf(hiveStoragePlugin.getHiveConf(), confProperties)),
         path.getParent());
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetScan.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetScan.java
index 617f6a5..79a07f1 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetScan.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetScan.java
@@ -114,7 +114,7 @@ public class HiveDrillNativeParquetScan extends AbstractParquetGroupScan {
       assert split instanceof FileSplit;
       FileSplit fileSplit = (FileSplit) split;
       Path finalPath = fileSplit.getPath();
-      String pathString = Path.getPathWithoutSchemeAndAuthority(finalPath).toString();
+      Path pathString = Path.getPathWithoutSchemeAndAuthority(finalPath);
       entries.add(new ReadEntryWithPath(pathString));
 
       // store partition values per path
@@ -205,7 +205,7 @@ public class HiveDrillNativeParquetScan extends AbstractParquetGroupScan {
   protected void initInternal() throws IOException {
     Map<FileStatus, FileSystem> fileStatusConfMap = new LinkedHashMap<>();
     for (ReadEntryWithPath entry : entries) {
-      Path path = new Path(entry.getPath());
+      Path path = entry.getPath();
       Configuration conf = new ProjectionPusher().pushProjectionsAndFilters(
           new JobConf(hiveStoragePlugin.getHiveConf()),
           path.getParent());
@@ -221,7 +221,7 @@ public class HiveDrillNativeParquetScan extends AbstractParquetGroupScan {
   }
 
   @Override
-  protected AbstractParquetGroupScan cloneWithFileSelection(Collection<String> filePaths) throws IOException {
+  protected AbstractParquetGroupScan cloneWithFileSelection(Collection<Path> filePaths) throws IOException {
     FileSelection newSelection = new FileSelection(null, new ArrayList<>(filePaths), null, null, false);
     return clone(newSelection);
   }
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetScanBatchCreator.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetScanBatchCreator.java
index 339e1bd..be906fe 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetScanBatchCreator.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetScanBatchCreator.java
@@ -28,6 +28,7 @@ import org.apache.drill.exec.server.options.OptionManager;
 import org.apache.drill.exec.store.dfs.DrillFileSystem;
 import org.apache.drill.exec.store.parquet.AbstractParquetScanBatchCreator;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
 
 import java.io.IOException;
 import java.util.HashMap;
@@ -54,7 +55,7 @@ public class HiveDrillNativeParquetScanBatchCreator extends AbstractParquetScanB
    */
   private class HiveDrillNativeParquetDrillFileSystemManager extends AbstractDrillFileSystemManager {
 
-    private final Map<String, DrillFileSystem> fileSystems;
+    private final Map<Path, DrillFileSystem> fileSystems;
 
     HiveDrillNativeParquetDrillFileSystemManager(OperatorContext operatorContext) {
       super(operatorContext);
@@ -62,7 +63,7 @@ public class HiveDrillNativeParquetScanBatchCreator extends AbstractParquetScanB
     }
 
     @Override
-    protected DrillFileSystem get(Configuration config, String path) throws ExecutionSetupException {
+    protected DrillFileSystem get(Configuration config, Path path) throws ExecutionSetupException {
       DrillFileSystem fs = fileSystems.get(path);
       if (fs == null) {
         try {
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HivePartitionHolder.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HivePartitionHolder.java
index 803144e..3e16acb 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HivePartitionHolder.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HivePartitionHolder.java
@@ -19,6 +19,7 @@ package org.apache.drill.exec.store.hive;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.hadoop.fs.Path;
 
 import java.util.ArrayList;
 import java.util.Collections;
@@ -34,11 +35,11 @@ import java.util.Map;
  */
 public class HivePartitionHolder {
 
-  private final Map<String, Integer> keyToIndexMapper;
+  private final Map<Path, Integer> keyToIndexMapper;
   private final List<List<String>> partitionValues;
 
   @JsonCreator
-  public HivePartitionHolder(@JsonProperty("keyToIndexMapper") Map<String, Integer> keyToIndexMapper,
+  public HivePartitionHolder(@JsonProperty("keyToIndexMapper") Map<Path, Integer> keyToIndexMapper,
                              @JsonProperty("partitionValues") List<List<String>> partitionValues) {
     this.keyToIndexMapper = keyToIndexMapper;
     this.partitionValues = partitionValues;
@@ -50,7 +51,7 @@ public class HivePartitionHolder {
   }
 
   @JsonProperty
-  public Map<String, Integer> getKeyToIndexMapper() {
+  public Map<Path, Integer> getKeyToIndexMapper() {
     return keyToIndexMapper;
   }
 
@@ -67,7 +68,7 @@ public class HivePartitionHolder {
    * @param key mapper key
    * @param values partition values
    */
-  public void add(String key, List<String> values) {
+  public void add(Path key, List<String> values) {
     int index = partitionValues.indexOf(values);
     if (index == -1) {
       index = partitionValues.size();
@@ -84,7 +85,7 @@ public class HivePartitionHolder {
    * @param key mapper key
    * @return list of partition values
    */
-  public List<String> get(String key) {
+  public List<String> get(Path key) {
     Integer index = keyToIndexMapper.get(key);
     if (index == null) {
       return Collections.emptyList();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/PhysicalPlan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/PhysicalPlan.java
index f73afb1..6eba658 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/PhysicalPlan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/PhysicalPlan.java
@@ -45,7 +45,8 @@ public class PhysicalPlan {
   Graph<PhysicalOperator, Root, Leaf> graph;
 
   @JsonCreator
-  public PhysicalPlan(@JsonProperty("head") PlanProperties properties, @JsonProperty("graph") List<PhysicalOperator> operators){
+  public PhysicalPlan(@JsonProperty("head") PlanProperties properties,
+                      @JsonProperty("graph") List<PhysicalOperator> operators) {
     this.properties = properties;
     this.graph = Graph.newGraph(operators, Root.class, Leaf.class);
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java
index 7e2623a..03b53a4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java
@@ -34,6 +34,7 @@ import org.apache.drill.exec.planner.physical.PlannerSettings;
 
 import com.fasterxml.jackson.annotation.JsonIgnore;
 import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.hadoop.fs.Path;
 
 public abstract class AbstractGroupScan extends AbstractBase implements GroupScan {
 
@@ -171,7 +172,7 @@ public abstract class AbstractGroupScan extends AbstractBase implements GroupSca
   }
 
   @Override
-  public Collection<String> getFiles() {
+  public Collection<Path> getFiles() {
     return null;
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java
index e42ae2d..6dbad22 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java
@@ -30,6 +30,7 @@ import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.server.options.OptionManager;
 import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import org.apache.hadoop.fs.Path;
 
 /**
  * A GroupScan operator represents all data which will be scanned by a given physical
@@ -142,8 +143,10 @@ public interface GroupScan extends Scan, HasAffinity{
   /**
    * Returns a collection of file names associated with this GroupScan. This should be called after checking
    * hasFiles().  If this GroupScan cannot provide file names, it returns null.
+   *
+   * @return collection of files paths
    */
-  Collection<String> getFiles();
+  Collection<Path> getFiles();
 
   @JsonIgnore
   LogicalExpression getFilter();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/SchemalessScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/SchemalessScan.java
index 999c417..fb60ddd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/SchemalessScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/SchemalessScan.java
@@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonTypeName;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.proto.CoordinationProtos;
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+import org.apache.hadoop.fs.Path;
 
 import java.util.List;
 
@@ -32,17 +33,17 @@ import java.util.List;
 @JsonTypeName("schemaless-scan")
 public class SchemalessScan extends AbstractFileGroupScan implements SubScan {
 
-  private final String selectionRoot;
+  private final Path selectionRoot;
 
   @JsonCreator
   public SchemalessScan(@JsonProperty("userName") String userName,
-                        @JsonProperty("selectionRoot") String selectionRoot,
+                        @JsonProperty("selectionRoot") Path selectionRoot,
                         @JsonProperty("columns") List<SchemaPath> columns) {
     this(userName, selectionRoot);
   }
 
   public SchemalessScan(@JsonProperty("userName") String userName,
-                        @JsonProperty("selectionRoot") String selectionRoot) {
+                        @JsonProperty("selectionRoot") Path selectionRoot) {
     super(userName);
     this.selectionRoot = selectionRoot;
   }
@@ -53,7 +54,7 @@ public class SchemalessScan extends AbstractFileGroupScan implements SubScan {
   }
 
   @JsonProperty
-  public String getSelectionRoot() {
+  public Path getSelectionRoot() {
     return selectionRoot;
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/BaseFileScanFramework.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/BaseFileScanFramework.java
index 5a8c526..8352dfa 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/BaseFileScanFramework.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/BaseFileScanFramework.java
@@ -110,7 +110,7 @@ public abstract class BaseFileScanFramework<T extends BaseFileScanFramework.File
 
     List<Path> paths = new ArrayList<>();
     for (FileWork work : files) {
-      Path path = dfs.makeQualified(new Path(work.getPath()));
+      Path path = dfs.makeQualified(work.getPath());
       paths.add(path);
       FileSplit split = new FileSplit(path, work.getStart(), work.getLength(), new String[]{""});
       spilts.add(split);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileMetadata.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileMetadata.java
index ff449d4..2eb8af9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileMetadata.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileMetadata.java
@@ -52,7 +52,7 @@ public class FileMetadata {
       return;
     }
 
-    dirPath = ColumnExplorer.parsePartitions(filePath.toString(), rootPath.toString());
+    dirPath = ColumnExplorer.parsePartitions(filePath, rootPath, false);
     if (dirPath == null) {
       throw new IllegalArgumentException(
           String.format("Selection root of \"%s\" is not a leading path of \"%s\"",
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/AbstractPartitionDescriptor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/AbstractPartitionDescriptor.java
index cf01256..d414827 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/AbstractPartitionDescriptor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/AbstractPartitionDescriptor.java
@@ -22,6 +22,7 @@ import java.util.List;
 
 import org.apache.calcite.rel.core.TableScan;
 import org.apache.drill.exec.store.dfs.MetadataContext;
+import org.apache.hadoop.fs.Path;
 
 /**
  * Abstract base class for file system based partition descriptors and Hive partition descriptors.
@@ -65,7 +66,7 @@ public abstract class AbstractPartitionDescriptor implements PartitionDescriptor
 
 
   @Override
-  public TableScan createTableScan(List<PartitionLocation> newPartitions, String cacheFileRoot,
+  public TableScan createTableScan(List<PartitionLocation> newPartitions, Path cacheFileRoot,
       boolean isAllPruned, MetadataContext metaContext) throws Exception {
     throw new UnsupportedOperationException();
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/DFSDirPartitionLocation.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/DFSDirPartitionLocation.java
index 46afb30..440178b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/DFSDirPartitionLocation.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/DFSDirPartitionLocation.java
@@ -21,7 +21,9 @@
 package org.apache.drill.exec.planner;
 
 
+import org.apache.drill.exec.util.DrillFileSystemUtil;
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.hadoop.fs.Path;
 
 import java.util.Collection;
 import java.util.List;
@@ -46,7 +48,7 @@ public class DFSDirPartitionLocation implements PartitionLocation {
   }
 
   @Override
-  public String getEntirePartitionLocation() {
+  public Path getEntirePartitionLocation() {
     throw new UnsupportedOperationException("Should not call getEntirePartitionLocation for composite partition location!");
   }
 
@@ -67,15 +69,17 @@ public class DFSDirPartitionLocation implements PartitionLocation {
   }
 
   @Override
-  public String getCompositePartitionPath() {
-    String path = "";
-    for (int i=0; i < dirs.length; i++) {
-      if (dirs[i] == null) { // get the prefix
+  public Path getCompositePartitionPath() {
+    StringBuilder path = new StringBuilder();
+    for (String dir : dirs) {
+      if (dir == null) { // get the prefix
         break;
       }
-      path += "/" + dirs[i];
+      path.append("/")
+          .append(dir);
     }
-    return path;
+
+    return DrillFileSystemUtil.createPathSafe(path.toString());
   }
 
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/DFSFilePartitionLocation.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/DFSFilePartitionLocation.java
index ecfa622..0dfe316 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/DFSFilePartitionLocation.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/DFSFilePartitionLocation.java
@@ -17,6 +17,7 @@
  */
 package org.apache.drill.exec.planner;
 
+import org.apache.drill.exec.store.ColumnExplorer;
 import org.apache.hadoop.fs.Path;
 
 /**
@@ -24,28 +25,14 @@ import org.apache.hadoop.fs.Path;
  */
 public class DFSFilePartitionLocation extends SimplePartitionLocation {
   private final String[] dirs;
-  private final String file;
+  private final Path file;
 
-  public DFSFilePartitionLocation(int max, String selectionRoot, String file, boolean hasDirsOnly) {
+  public DFSFilePartitionLocation(int max, Path selectionRoot, Path file, boolean hasDirsOnly) {
     this.file = file;
     this.dirs = new String[max];
 
-    // strip the scheme and authority if they exist
-    selectionRoot = Path.getPathWithoutSchemeAndAuthority(new Path(selectionRoot)).toString();
-
-    int start = file.indexOf(selectionRoot) + selectionRoot.length();
-    String postPath = file.substring(start);
-    if (postPath.length() == 0) {
-      return;
-    }
-    if(postPath.charAt(0) == '/'){
-      postPath = postPath.substring(1);
-    }
-    String[] mostDirs = postPath.split("/");
-    int maxLoop = Math.min(max, hasDirsOnly ? mostDirs.length : mostDirs.length - 1);
-    for(int i =0; i < maxLoop; i++) {
-      this.dirs[i] = mostDirs[i];
-    }
+    String[] dirs = ColumnExplorer.parsePartitions(this.file, selectionRoot, hasDirsOnly);
+    System.arraycopy(dirs, 0, this.dirs, 0, Math.min(max, dirs.length));
   }
 
   /**
@@ -64,7 +51,7 @@ public class DFSFilePartitionLocation extends SimplePartitionLocation {
    * @return The partition location.
    */
   @Override
-  public String getEntirePartitionLocation() {
+  public Path getEntirePartitionLocation() {
     return file;
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FileSystemPartitionDescriptor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FileSystemPartitionDescriptor.java
index 6526892..1880490 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FileSystemPartitionDescriptor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FileSystemPartitionDescriptor.java
@@ -26,6 +26,8 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.drill.common.util.GuavaUtils;
 import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
@@ -37,7 +39,6 @@ import org.apache.calcite.adapter.enumerable.EnumerableTableScan;
 import org.apache.calcite.prepare.RelOptTableImpl;
 import org.apache.calcite.rel.core.TableScan;
 import org.apache.calcite.util.BitSets;
-import org.apache.calcite.util.Pair;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.Types;
@@ -54,6 +55,7 @@ import org.apache.drill.exec.store.dfs.FormatSelection;
 import org.apache.drill.exec.store.dfs.MetadataContext;
 import org.apache.drill.exec.vector.NullableVarCharVector;
 import org.apache.drill.exec.vector.ValueVector;
+import org.apache.hadoop.fs.Path;
 
 
 // partition descriptor for file system based tables
@@ -145,18 +147,18 @@ public class FileSystemPartitionDescriptor extends AbstractPartitionDescriptor {
   }
 
   @Override
-  public String getBaseTableLocation() {
+  public Path getBaseTableLocation() {
     final FormatSelection origSelection = (FormatSelection) table.getSelection();
-    return origSelection.getSelection().selectionRoot;
+    return origSelection.getSelection().getSelectionRoot();
   }
 
   @Override
   protected void createPartitionSublists() {
-    final Pair<Collection<String>, Boolean> fileLocationsAndStatus = getFileLocationsAndStatus();
+    final Pair<Collection<Path>, Boolean> fileLocationsAndStatus = getFileLocationsAndStatus();
     List<PartitionLocation> locations = new LinkedList<>();
-    boolean hasDirsOnly = fileLocationsAndStatus.right;
+    boolean hasDirsOnly = fileLocationsAndStatus.getRight();
 
-    final String selectionRoot = getBaseTableLocation();
+    final Path selectionRoot = getBaseTableLocation();
 
     // map used to map the partition keys (dir0, dir1, ..), to the list of partitions that share the same partition keys.
     // For example,
@@ -166,35 +168,31 @@ public class FileSystemPartitionDescriptor extends AbstractPartitionDescriptor {
 
     // Figure out the list of leaf subdirectories. For each leaf subdirectory, find the list of files (DFSFilePartitionLocation)
     // it contains.
-    for (String file: fileLocationsAndStatus.left) {
-      DFSFilePartitionLocation dfsFilePartitionLocation = new DFSFilePartitionLocation(MAX_NESTED_SUBDIRS, selectionRoot, file, hasDirsOnly);
+    for (Path file: fileLocationsAndStatus.getLeft()) {
+      DFSFilePartitionLocation dfsFilePartitionLocation = new DFSFilePartitionLocation(MAX_NESTED_SUBDIRS,
+          selectionRoot, file, hasDirsOnly);
+      List<String> dirList = Arrays.asList(dfsFilePartitionLocation.getDirs());
 
-      final String[] dirs = dfsFilePartitionLocation.getDirs();
-      final List<String> dirList = Arrays.asList(dirs);
-
-      if (!dirToFileMap.containsKey(dirList)) {
-        dirToFileMap.put(dirList, new ArrayList<PartitionLocation>());
-      }
+      dirToFileMap.putIfAbsent(dirList, new ArrayList<>());
       dirToFileMap.get(dirList).add(dfsFilePartitionLocation);
     }
 
     // build a list of DFSDirPartitionLocation.
-    for (final List<String> dirs : dirToFileMap.keySet()) {
-      locations.add( new DFSDirPartitionLocation(dirs.toArray(new String[dirs.size()]), dirToFileMap.get(dirs)));
-    }
+    dirToFileMap.keySet().stream()
+        .map(dirs -> new DFSDirPartitionLocation(dirs.toArray(new String[dirs.size()]), dirToFileMap.get(dirs)))
+        .forEach(locations::add);
 
     locationSuperList = Lists.partition(locations, PartitionDescriptor.PARTITION_BATCH_SIZE);
     sublistsCreated = true;
   }
 
-  protected Pair<Collection<String>, Boolean> getFileLocationsAndStatus() {
-    Collection<String> fileLocations = null;
-    Pair<Collection<String>, Boolean> fileLocationsAndStatus = null;
+  protected Pair<Collection<Path>, Boolean> getFileLocationsAndStatus() {
+    Collection<Path> fileLocations = null;
     boolean isExpandedPartial = false;
     if (scanRel instanceof DrillScanRel) {
       // If a particular GroupScan provides files, get the list of files from there rather than
       // DrillTable because GroupScan would have the updated version of the selection
-      final DrillScanRel drillScan = (DrillScanRel) scanRel;
+      DrillScanRel drillScan = (DrillScanRel) scanRel;
       if (drillScan.getGroupScan().hasFiles()) {
         fileLocations = drillScan.getGroupScan().getFiles();
         isExpandedPartial = false;
@@ -208,67 +206,59 @@ public class FileSystemPartitionDescriptor extends AbstractPartitionDescriptor {
       fileLocations = selection.getFiles();
       isExpandedPartial = selection.isExpandedPartial();
     }
-    fileLocationsAndStatus = Pair.of(fileLocations, isExpandedPartial);
-    return fileLocationsAndStatus;
+    return Pair.of(fileLocations, isExpandedPartial);
   }
 
   @Override
-  public TableScan createTableScan(List<PartitionLocation> newPartitionLocation, String cacheFileRoot,
+  public TableScan createTableScan(List<PartitionLocation> newPartitionLocation, Path cacheFileRoot,
       boolean wasAllPartitionsPruned, MetadataContext metaContext) throws Exception {
-    List<String> newFiles = Lists.newArrayList();
-    for (final PartitionLocation location : newPartitionLocation) {
+    List<Path> newFiles = new ArrayList<>();
+    for (PartitionLocation location : newPartitionLocation) {
       if (!location.isCompositePartition()) {
         newFiles.add(location.getEntirePartitionLocation());
       } else {
         final Collection<SimplePartitionLocation> subPartitions = location.getPartitionLocationRecursive();
-        for (final PartitionLocation subPart : subPartitions) {
+        for (PartitionLocation subPart : subPartitions) {
           newFiles.add(subPart.getEntirePartitionLocation());
         }
       }
     }
 
+    FormatSelection formatSelection = (FormatSelection) table.getSelection();
+    FileSelection newFileSelection = new FileSelection(null, newFiles, getBaseTableLocation(),
+        cacheFileRoot, wasAllPartitionsPruned, formatSelection.getSelection().getDirStatus());
+    newFileSelection.setMetaContext(metaContext);
+    RelOptTable relOptTable = scanRel.getTable();
+
     if (scanRel instanceof DrillScanRel) {
-      final FormatSelection formatSelection = (FormatSelection)table.getSelection();
-      final FileSelection newFileSelection = new FileSelection(null, newFiles, getBaseTableLocation(),
-          cacheFileRoot, wasAllPartitionsPruned, formatSelection.getSelection().getDirStatus());
-      newFileSelection.setMetaContext(metaContext);
-      final FileGroupScan newGroupScan =
-          ((FileGroupScan)((DrillScanRel)scanRel).getGroupScan()).clone(newFileSelection);
+      FileGroupScan newGroupScan =
+          ((FileGroupScan) ((DrillScanRel) scanRel).getGroupScan()).clone(newFileSelection);
       return new DrillScanRel(scanRel.getCluster(),
                       scanRel.getTraitSet().plus(DrillRel.DRILL_LOGICAL),
-                      scanRel.getTable(),
+                      relOptTable,
                       newGroupScan,
                       scanRel.getRowType(),
                       ((DrillScanRel) scanRel).getColumns(),
                       true /*filter pushdown*/);
     } else if (scanRel instanceof EnumerableTableScan) {
-      return createNewTableScanFromSelection((EnumerableTableScan)scanRel, newFiles, cacheFileRoot,
-          wasAllPartitionsPruned, metaContext);
+      FormatSelection newFormatSelection = new FormatSelection(formatSelection.getFormat(), newFileSelection);
+
+      DynamicDrillTable dynamicDrillTable = new DynamicDrillTable(table.getPlugin(), table.getStorageEngineName(),
+          table.getUserName(), newFormatSelection);
+      /* Copy statistics from the original relOptTable */
+      dynamicDrillTable.setStatsTable(table.getStatsTable());
+      DrillTranslatableTable newTable = new DrillTranslatableTable(dynamicDrillTable);
+
+      RelOptTableImpl newOptTableImpl = RelOptTableImpl.create(relOptTable.getRelOptSchema(), relOptTable.getRowType(),
+          newTable, GuavaUtils.convertToUnshadedImmutableList(ImmutableList.of()));
+
+      // return an EnumerableTableScan with fileSelection being part of digest of TableScan node.
+      return DirPrunedEnumerableTableScan.create(scanRel.getCluster(), newOptTableImpl, newFileSelection.toString());
     } else {
       throw new UnsupportedOperationException("Only DrillScanRel and EnumerableTableScan is allowed!");
     }
   }
 
-  private TableScan createNewTableScanFromSelection(EnumerableTableScan oldScan, List<String> newFiles, String cacheFileRoot,
-      boolean wasAllPartitionsPruned, MetadataContext metaContext) {
-    final RelOptTableImpl t = (RelOptTableImpl) oldScan.getTable();
-    final FormatSelection formatSelection = (FormatSelection) table.getSelection();
-    final FileSelection newFileSelection = new FileSelection(null, newFiles, getBaseTableLocation(),
-            cacheFileRoot, wasAllPartitionsPruned, formatSelection.getSelection().getDirStatus());
-    newFileSelection.setMetaContext(metaContext);
-    final FormatSelection newFormatSelection = new FormatSelection(formatSelection.getFormat(), newFileSelection);
-    final DynamicDrillTable dynamicDrillTable = new DynamicDrillTable(table.getPlugin(), table.getStorageEngineName(),
-            table.getUserName(), newFormatSelection);
-    /* Copy statistics from the original table */
-    dynamicDrillTable.setStatsTable(table.getStatsTable());
-    final DrillTranslatableTable newTable = new DrillTranslatableTable(dynamicDrillTable);
-    final RelOptTableImpl newOptTableImpl = RelOptTableImpl.create(t.getRelOptSchema(), t.getRowType(), newTable,
-            GuavaUtils.convertToUnshadedImmutableList(ImmutableList.of()));
-
-    // return an EnumerableTableScan with fileSelection being part of digest of TableScan node.
-    return DirPrunedEnumerableTableScan.create(oldScan.getCluster(), newOptTableImpl, newFileSelection.toString());
-  }
-
   @Override
   public TableScan createTableScan(List<PartitionLocation> newPartitionLocation,
       boolean wasAllPartitionsPruned) throws Exception {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ParquetPartitionDescriptor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ParquetPartitionDescriptor.java
index df6decc..23675d1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ParquetPartitionDescriptor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ParquetPartitionDescriptor.java
@@ -134,17 +134,17 @@ public class ParquetPartitionDescriptor extends AbstractPartitionDescriptor {
   }
 
   @Override
-  public String getBaseTableLocation() {
+  public Path getBaseTableLocation() {
     final FormatSelection origSelection = (FormatSelection) scanRel.getDrillTable().getSelection();
     return origSelection.getSelection().selectionRoot;
   }
 
   @Override
   public TableScan createTableScan(List<PartitionLocation> newPartitionLocation,
-                                   String cacheFileRoot,
+                                   Path cacheFileRoot,
                                    boolean wasAllPartitionsPruned,
                                    MetadataContext metaContext) throws Exception {
-    List<String> newFiles = new ArrayList<>();
+    List<Path> newFiles = new ArrayList<>();
     for (final PartitionLocation location : newPartitionLocation) {
       newFiles.add(location.getEntirePartitionLocation());
     }
@@ -172,17 +172,17 @@ public class ParquetPartitionDescriptor extends AbstractPartitionDescriptor {
 
   @Override
   protected void createPartitionSublists() {
-    Set<String> fileLocations = groupScan.getFileSet();
+    Set<Path> fileLocations = groupScan.getFileSet();
     List<PartitionLocation> locations = new LinkedList<>();
-    for (String file : fileLocations) {
+    for (Path file : fileLocations) {
       locations.add(new ParquetPartitionLocation(file));
     }
     locationSuperList = Lists.partition(locations, PartitionDescriptor.PARTITION_BATCH_SIZE);
     sublistsCreated = true;
   }
 
-  private GroupScan createNewGroupScan(List<String> newFiles,
-                                       String cacheFileRoot,
+  private GroupScan createNewGroupScan(List<Path> newFiles,
+                                       Path cacheFileRoot,
                                        boolean wasAllPartitionsPruned,
                                        MetadataContext metaContext) throws IOException {
 
@@ -194,8 +194,8 @@ public class ParquetPartitionDescriptor extends AbstractPartitionDescriptor {
     return groupScan.clone(newSelection);
   }
 
-  private void populatePruningVector(ValueVector v, int index, SchemaPath column, String file) {
-    String path = Path.getPathWithoutSchemeAndAuthority(new Path(file)).toString();
+  private void populatePruningVector(ValueVector v, int index, SchemaPath column, Path file) {
+    Path path = Path.getPathWithoutSchemeAndAuthority(file);
     TypeProtos.MajorType majorType = getVectorType(column, null);
     TypeProtos.MinorType type = majorType.getMinorType();
     switch (type) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ParquetPartitionLocation.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ParquetPartitionLocation.java
index 49e5109..eec0d5e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ParquetPartitionLocation.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ParquetPartitionLocation.java
@@ -17,6 +17,8 @@
  */
 package org.apache.drill.exec.planner;
 
+import org.apache.hadoop.fs.Path;
+
 /*
  * PartitionLocation for the parquet auto partitioning scheme. We just store
  * the location of each partition within this class. Since the partition value
@@ -25,9 +27,9 @@ package org.apache.drill.exec.planner;
  * invoked.
  */
 public class ParquetPartitionLocation extends SimplePartitionLocation {
-  private final String file;
+  private final Path file;
 
-  public ParquetPartitionLocation(String file) {
+  public ParquetPartitionLocation(Path file) {
     this.file = file;
   }
 
@@ -48,7 +50,7 @@ public class ParquetPartitionLocation extends SimplePartitionLocation {
    * @return String location of the partition
    */
   @Override
-  public String getEntirePartitionLocation() {
+  public Path getEntirePartitionLocation() {
     return file;
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PartitionDescriptor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PartitionDescriptor.java
index 220bf29..8759078 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PartitionDescriptor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PartitionDescriptor.java
@@ -23,36 +23,52 @@ import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.store.dfs.MetadataContext;
 import org.apache.drill.exec.vector.ValueVector;
+import org.apache.hadoop.fs.Path;
 
 import java.util.BitSet;
 import java.util.List;
 import java.util.Map;
 
-// Interface used to describe partitions. Currently used by file system based partitions and hive partitions
+/**
+ * Interface used to describe partitions. Currently used by file system based partitions and hive partitions
+ */
 public interface PartitionDescriptor extends Iterable<List<PartitionLocation>> {
 
-  public static final int PARTITION_BATCH_SIZE = Character.MAX_VALUE;
+  int PARTITION_BATCH_SIZE = Character.MAX_VALUE;
 
-  /* Get the hierarchy index of the given partition
+  /**
+   * Get the hierarchy index of the given partition
    * For eg: if we have the partition laid out as follows
    * 1997/q1/jan
-   *
    * then getPartitionHierarchyIndex("jan") => 2
+   *
+   * @param partitionName Partition name
+   * @return the index of specified partition name in the hierarchy
    */
-  public int getPartitionHierarchyIndex(String partitionName);
+  int getPartitionHierarchyIndex(String partitionName);
 
-  // Given a column name return boolean to indicate if its a partition column or not
-  public boolean isPartitionName(String name);
+  /**
+   * Given a column name return boolean to indicate if its a partition column or not
+   *
+   * @param name of Partition
+   * @return true, if this is the partition name and vise versa.
+   */
+  boolean isPartitionName(String name);
 
   /**
    * Check to see if the name is a partition name.
+   *
    * @param name The field name you want to compare to partition names.
    * @return Return index if valid, otherwise return null;
    */
-  public Integer getIdIfValid(String name);
+  Integer getIdIfValid(String name);
 
-  // Maximum level of partition nesting/ hierarchy supported
-  public int getMaxHierarchyLevel();
+  /**
+   * Maximum level of partition nesting/ hierarchy supported
+   *
+   * @return maximum supported level number of partition hierarchy
+   */
+  int getMaxHierarchyLevel();
 
   /**
    * Method creates an in memory representation of all the partitions. For each level of partitioning we
@@ -79,7 +95,7 @@ public interface PartitionDescriptor extends Iterable<List<PartitionLocation>> {
    * @param wasAllPartitionsPruned
    * @throws Exception
    */
-  public TableScan createTableScan(List<PartitionLocation> newPartitions,
+  TableScan createTableScan(List<PartitionLocation> newPartitions,
       boolean wasAllPartitionsPruned) throws Exception;
 
   /**
@@ -91,11 +107,11 @@ public interface PartitionDescriptor extends Iterable<List<PartitionLocation>> {
    * @param metaContext
    * @throws Exception
    */
-  public TableScan createTableScan(List<PartitionLocation> newPartitions, String cacheFileRoot,
+  TableScan createTableScan(List<PartitionLocation> newPartitions, Path cacheFileRoot,
       boolean wasAllPartitionsPruned, MetadataContext metaContext) throws Exception;
 
-  public boolean supportsMetadataCachePruning();
+  boolean supportsMetadataCachePruning();
 
-  public String getBaseTableLocation();
+  Path getBaseTableLocation();
 
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PartitionLocation.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PartitionLocation.java
index 22088f7..2ef4e17 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PartitionLocation.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PartitionLocation.java
@@ -17,6 +17,8 @@
  */
 package org.apache.drill.exec.planner;
 
+import org.apache.hadoop.fs.Path;
+
 import java.util.List;
 
 /**
@@ -37,27 +39,27 @@ public interface PartitionLocation {
   /**
    * Returns the value of the 'index' partition column
    */
-  public String getPartitionValue(int index);
+  String getPartitionValue(int index);
 
   /**
-   * Returns the string representation of this partition.
+   * Returns the path of this partition.
    * Only a non-composite partition supports this.
    */
-  public String getEntirePartitionLocation();
+  Path getEntirePartitionLocation();
 
   /**
    * Returns the list of the non-composite partitions that this partition consists of.
    */
-  public List<SimplePartitionLocation> getPartitionLocationRecursive();
+  List<SimplePartitionLocation> getPartitionLocationRecursive();
 
   /**
    * Returns if this is a simple or composite partition.
    */
-  public boolean isCompositePartition();
+  boolean isCompositePartition();
 
   /**
    * Returns the path string of directory names only for composite partition
    */
-  public String getCompositePartitionPath();
+  Path getCompositePartitionPath();
 
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PhysicalPlanReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PhysicalPlanReader.java
index 20b06c4..920f9b2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PhysicalPlanReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PhysicalPlanReader.java
@@ -33,6 +33,7 @@ import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.PhysicalOperatorUtil;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.record.MajorTypeSerDe;
+import org.apache.drill.exec.serialization.PathSerDe;
 import org.apache.drill.exec.server.options.OptionList;
 import org.apache.drill.exec.store.StoragePluginRegistry;
 import org.apache.drill.exec.store.pojo.DynamicPojoRecordReader;
@@ -44,43 +45,43 @@ import com.fasterxml.jackson.databind.ObjectReader;
 import com.fasterxml.jackson.databind.deser.std.StdDelegatingDeserializer;
 import com.fasterxml.jackson.databind.module.SimpleModule;
 import org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.fs.Path;
 
 public class PhysicalPlanReader {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PhysicalPlanReader.class);
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PhysicalPlanReader.class);
 
   private final ObjectReader physicalPlanReader;
   private final ObjectMapper mapper;
   private final ObjectReader operatorReader;
   private final ObjectReader logicalPlanReader;
 
-  public PhysicalPlanReader(DrillConfig config, ScanResult scanResult, LogicalPlanPersistence lpPersistance, final DrillbitEndpoint endpoint,
-                            final StoragePluginRegistry pluginRegistry) {
+  public PhysicalPlanReader(DrillConfig config, ScanResult scanResult, LogicalPlanPersistence lpPersistance,
+                            final DrillbitEndpoint endpoint, final StoragePluginRegistry pluginRegistry) {
 
     ObjectMapper lpMapper = lpPersistance.getMapper();
 
     // Endpoint serializer/deserializer.
-    SimpleModule deserModule = new SimpleModule("PhysicalOperatorModule") //
-        .addSerializer(DrillbitEndpoint.class, new DrillbitEndpointSerDe.Se()) //
-        .addDeserializer(DrillbitEndpoint.class, new DrillbitEndpointSerDe.De()) //
+    SimpleModule serDeModule = new SimpleModule("PhysicalOperatorModule")
+        .addSerializer(DrillbitEndpoint.class, new DrillbitEndpointSerDe.Se())
+        .addDeserializer(DrillbitEndpoint.class, new DrillbitEndpointSerDe.De())
         .addSerializer(MajorType.class, new MajorTypeSerDe.Se())
         .addDeserializer(MajorType.class, new MajorTypeSerDe.De())
         .addDeserializer(DynamicPojoRecordReader.class,
-            new StdDelegatingDeserializer<>(new DynamicPojoRecordReader.Converter(lpMapper)));
+            new StdDelegatingDeserializer<>(new DynamicPojoRecordReader.Converter(lpMapper)))
+        .addSerializer(Path.class, new PathSerDe.Se());
 
-    lpMapper.registerModule(deserModule);
+    lpMapper.registerModule(serDeModule);
     Set<Class<? extends PhysicalOperator>> subTypes = PhysicalOperatorUtil.getSubTypes(scanResult);
-    for (Class<? extends PhysicalOperator> subType : subTypes) {
-      lpMapper.registerSubtypes(subType);
-    }
+    subTypes.forEach(lpMapper::registerSubtypes);
     lpMapper.registerSubtypes(DynamicPojoRecordReader.class);
-    InjectableValues injectables = new InjectableValues.Std() //
-            .addValue(StoragePluginRegistry.class, pluginRegistry) //
-        .addValue(DrillbitEndpoint.class, endpoint); //
+    InjectableValues injectables = new InjectableValues.Std()
+        .addValue(StoragePluginRegistry.class, pluginRegistry)
+        .addValue(DrillbitEndpoint.class, endpoint);
 
     this.mapper = lpMapper;
-    this.physicalPlanReader = mapper.reader(PhysicalPlan.class).with(injectables);
-    this.operatorReader = mapper.reader(PhysicalOperator.class).with(injectables);
-    this.logicalPlanReader = mapper.reader(LogicalPlan.class).with(injectables);
+    this.physicalPlanReader = mapper.readerFor(PhysicalPlan.class).with(injectables);
+    this.operatorReader = mapper.readerFor(PhysicalOperator.class).with(injectables);
+    this.logicalPlanReader = mapper.readerFor(LogicalPlan.class).with(injectables);
   }
 
   public String writeJson(OptionList list) throws JsonProcessingException{
@@ -91,33 +92,35 @@ public class PhysicalPlanReader {
     return mapper.writeValueAsString(op);
   }
 
-  public PhysicalPlan readPhysicalPlan(String json) throws JsonProcessingException, IOException {
+  public PhysicalPlan readPhysicalPlan(String json) throws IOException {
     logger.debug("Reading physical plan {}", json);
     return physicalPlanReader.readValue(json);
   }
 
-  public FragmentRoot readFragmentRoot(String json) throws JsonProcessingException, IOException {
+  public FragmentRoot readFragmentRoot(String json) throws IOException {
     logger.debug("Attempting to read {}", json);
     PhysicalOperator op = operatorReader.readValue(json);
-    if(op instanceof FragmentRoot){
+    if (op instanceof FragmentRoot) {
       return (FragmentRoot) op;
-    }else{
-      throw new UnsupportedOperationException(String.format("The provided json fragment doesn't have a FragmentRoot as its root operator.  The operator was %s.", op.getClass().getCanonicalName()));
+    } else {
+      throw new UnsupportedOperationException(String.format("The provided json fragment doesn't have a FragmentRoot " +
+          "as its root operator.  The operator was %s.", op.getClass().getCanonicalName()));
     }
   }
 
   @VisibleForTesting
-  public FragmentLeaf readFragmentLeaf(String json) throws JsonProcessingException, IOException {
+  public FragmentLeaf readFragmentLeaf(String json) throws IOException {
     logger.debug("Attempting to read {}", json);
     PhysicalOperator op = operatorReader.readValue(json);
     if (op instanceof FragmentLeaf){
       return (FragmentLeaf) op;
     } else {
-      throw new UnsupportedOperationException(String.format("The provided json fragment is not a FragmentLeaf. The operator was %s.", op.getClass().getCanonicalName()));
+      throw new UnsupportedOperationException(String.format("The provided json fragment is not a FragmentLeaf. " +
+          "The operator was %s.", op.getClass().getCanonicalName()));
     }
   }
 
-  public LogicalPlan readLogicalPlan(String json) throws JsonProcessingException, IOException{
+  public LogicalPlan readLogicalPlan(String json) throws IOException{
     logger.debug("Reading logical plan {}", json);
     return logicalPlanReader.readValue(json);
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/SimplePartitionLocation.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/SimplePartitionLocation.java
index 7c9afb0..ca00446 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/SimplePartitionLocation.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/SimplePartitionLocation.java
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.planner;
 
 import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import org.apache.hadoop.fs.Path;
 
 import java.util.List;
 
@@ -26,14 +27,14 @@ import java.util.List;
  * location of the entire partition and also stores the
  * value of the individual partition keys for this partition.
  */
-public abstract  class SimplePartitionLocation implements PartitionLocation{
+public abstract class SimplePartitionLocation implements PartitionLocation{
   @Override
   public boolean isCompositePartition() {
     return false;
   }
 
   @Override
-  public String getCompositePartitionPath() {
+  public Path getCompositePartitionPath() {
     throw new UnsupportedOperationException();
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/partition/PruneScanRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/partition/PruneScanRule.java
index 6f48b85..8d3a8ce 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/partition/PruneScanRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/partition/PruneScanRule.java
@@ -23,6 +23,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.drill.exec.util.DrillFileSystemUtil;
 import org.apache.drill.shaded.guava.com.google.common.base.Stopwatch;
 import org.apache.calcite.adapter.enumerable.EnumerableTableScan;
 import org.apache.calcite.rel.core.Filter;
@@ -74,7 +75,7 @@ import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
 import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
 
 import org.apache.drill.exec.vector.ValueVector;
-
+import org.apache.hadoop.fs.Path;
 
 public abstract class PruneScanRule extends StoragePluginOptimizerRule {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PruneScanRule.class);
@@ -372,7 +373,7 @@ public abstract class PruneScanRule extends StoragePluginOptimizerRule {
       // handle the case all partitions are filtered out.
       boolean canDropFilter = true;
       boolean wasAllPartitionsPruned = false;
-      String cacheFileRoot = null;
+      Path cacheFileRoot = null;
 
       if (newPartitions.isEmpty()) {
         assert firstLocation != null;
@@ -388,7 +389,7 @@ public abstract class PruneScanRule extends StoragePluginOptimizerRule {
 
         // set the cacheFileRoot appropriately
         if (firstLocation.isCompositePartition()) {
-          cacheFileRoot = descriptor.getBaseTableLocation() + firstLocation.getCompositePartitionPath();
+          cacheFileRoot = Path.mergePaths(descriptor.getBaseTableLocation(), firstLocation.getCompositePartitionPath());
         }
       }
 
@@ -408,7 +409,7 @@ public abstract class PruneScanRule extends StoragePluginOptimizerRule {
         // if metadata cache file could potentially be used, then assign a proper cacheFileRoot
         int index = -1;
         if (!matchBitSet.isEmpty()) {
-          String path = "";
+          StringBuilder path = new StringBuilder();
           index = matchBitSet.length() - 1;
 
           for (int j = 0; j < matchBitSet.length(); j++) {
@@ -419,10 +420,12 @@ public abstract class PruneScanRule extends StoragePluginOptimizerRule {
               break;
             }
           }
-          for (int j=0; j <= index; j++) {
-            path += "/" + spInfo[j];
+          for (int j = 0; j <= index; j++) {
+            path.append("/")
+                .append(spInfo[j]);
           }
-          cacheFileRoot = descriptor.getBaseTableLocation() + path;
+          cacheFileRoot = Path.mergePaths(descriptor.getBaseTableLocation(),
+              DrillFileSystemUtil.createPathSafe(path.toString()));
         }
         if (index != maxIndex) {
           // if multiple partitions are being selected, we should not drop the filter
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/AnalyzeTableHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/AnalyzeTableHandler.java
index f1acc71..8f62713 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/AnalyzeTableHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/AnalyzeTableHandler.java
@@ -133,15 +133,13 @@ public class AnalyzeTableHandler extends DefaultSqlHandler {
       DrillFileSystem fs = new DrillFileSystem(plugin.getFormatPlugin(
           formatSelection.getFormat()).getFsConf());
 
-      String selectionRoot = formatSelection.getSelection().getSelectionRoot();
-      if (!selectionRoot.contains(tableName)
-          || !fs.getFileStatus(new Path(selectionRoot)).isDirectory()) {
+      Path selectionRoot = formatSelection.getSelection().getSelectionRoot();
+      if (!selectionRoot.getName().equals(tableName) || !fs.getFileStatus(selectionRoot).isDirectory()) {
         return DrillStatsTable.notSupported(context, tableName);
       }
       // Do not recompute statistics, if stale
-      Path statsFilePath = new Path(new Path(selectionRoot), DotDrillType.STATS.getEnding());
-      if (fs.exists(statsFilePath)
-          && !isStatsStale(fs, statsFilePath)) {
+      Path statsFilePath = new Path(selectionRoot, DotDrillType.STATS.getEnding());
+      if (fs.exists(statsFilePath) && !isStatsStale(fs, statsFilePath)) {
        return DrillStatsTable.notRequired(context, tableName);
       }
     }
@@ -165,13 +163,8 @@ public class AnalyzeTableHandler extends DefaultSqlHandler {
     Path parentPath = statsFilePath.getParent();
     FileStatus directoryStatus = fs.getFileStatus(parentPath);
     // Parent directory modified after stats collection?
-    if (directoryStatus.getModificationTime() > statsFileModifyTime) {
-      return true;
-    }
-    if (tableModified(fs, parentPath, statsFileModifyTime)) {
-      return true;
-    }
-    return false;
+    return directoryStatus.getModificationTime() > statsFileModifyTime ||
+        tableModified(fs, parentPath, statsFileModifyTime);
   }
 
   /* Determines if the table was modified after computing statistics based on
@@ -185,10 +178,8 @@ public class AnalyzeTableHandler extends DefaultSqlHandler {
         return true;
       }
       // For a directory, we should recursively check sub-directories
-      if (file.isDirectory()) {
-        if (tableModified(fs, file.getPath(), statsModificationTime)) {
-          return true;
-        }
+      if (file.isDirectory() && tableModified(fs, file.getPath(), statsModificationTime)) {
+        return true;
       }
     }
     return false;
@@ -215,8 +206,7 @@ public class AnalyzeTableHandler extends DefaultSqlHandler {
 
   /* Converts to Drill logical plan */
   protected DrillRel convertToDrel(RelNode relNode, AbstractSchema schema, String analyzeTableName,
-      double samplePercent)
-      throws RelConversionException, SqlUnsupportedException {
+      double samplePercent) throws SqlUnsupportedException {
     DrillRel convertedRelNode = convertToRawDrel(relNode);
 
     if (convertedRelNode instanceof DrillStoreRel) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/RefreshMetadataHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/RefreshMetadataHandler.java
index 4684251..3ca7787 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/RefreshMetadataHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/RefreshMetadataHandler.java
@@ -108,8 +108,8 @@ public class RefreshMetadataHandler extends DefaultSqlHandler {
       FileSystemPlugin plugin = (FileSystemPlugin) drillTable.getPlugin();
       DrillFileSystem fs = new DrillFileSystem(plugin.getFormatPlugin(formatSelection.getFormat()).getFsConf());
 
-      String selectionRoot = formatSelection.getSelection().selectionRoot;
-      if (!fs.getFileStatus(new Path(selectionRoot)).isDirectory()) {
+      Path selectionRoot = formatSelection.getSelection().getSelectionRoot();
+      if (!fs.getFileStatus(selectionRoot).isDirectory()) {
         return notSupported(tableName);
       }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/ReadEntryWithPath.java b/exec/java-exec/src/main/java/org/apache/drill/exec/serialization/PathSerDe.java
similarity index 55%
copy from exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/ReadEntryWithPath.java
copy to exec/java-exec/src/main/java/org/apache/drill/exec/serialization/PathSerDe.java
index 4564e15..03bb37e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/ReadEntryWithPath.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/serialization/PathSerDe.java
@@ -15,28 +15,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.store.dfs;
+package org.apache.drill.exec.serialization;
 
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonSerializer;
+import com.fasterxml.jackson.databind.SerializerProvider;
+import org.apache.hadoop.fs.Path;
 
-public class ReadEntryWithPath {
+import java.io.IOException;
 
-  protected String path;
-
-
-  public ReadEntryWithPath(String path) {
-    super();
-    this.path = path;
-  }
-
-  public ReadEntryWithPath(){}
+/**
+ * Path serializer to simple String path. Without it the hadoop Path serialization creates a big JSON object.
+ */
+public class PathSerDe {
 
-  public String getPath(){
-   return path;
-  }
+  public static class Se extends JsonSerializer<Path> {
 
-  @Override
-  public String toString() {
-    return "ReadEntryWithPath [path=" + path + "]";
+    @Override
+    public void serialize(Path value, JsonGenerator gen, SerializerProvider serializers) throws IOException {
+      gen.writeString(value.toUri().getPath());
+    }
   }
-
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ColumnExplorer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ColumnExplorer.java
index d5d9784..33b5000 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ColumnExplorer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ColumnExplorer.java
@@ -17,6 +17,7 @@
  */
 package org.apache.drill.exec.store;
 
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.LinkedHashMap;
@@ -133,20 +134,20 @@ public class ColumnExplorer {
   public static List<String> getPartitionColumnNames(FileSelection selection, SchemaConfig schemaConfig) {
     int partitionsCount = 0;
     // a depth of table root path
-    int rootDepth = new Path(selection.getSelectionRoot()).depth();
+    int rootDepth = selection.getSelectionRoot().depth();
 
-    for (String file : selection.getFiles()) {
+    for (Path file : selection.getFiles()) {
       // Calculates partitions count for the concrete file:
       // depth of file path - depth of table root path - 1.
       // The depth of file path includes file itself,
       // so we should subtract 1 to consider only directories.
-      int currentPartitionsCount = new Path(file).depth() - rootDepth - 1;
+      int currentPartitionsCount = file.depth() - rootDepth - 1;
       // max depth of files path should be used to handle all partitions
       partitionsCount = Math.max(partitionsCount, currentPartitionsCount);
     }
 
     String partitionColumnLabel = schemaConfig.getOption(ExecConstants.FILESYSTEM_PARTITION_COLUMN_LABEL).string_val;
-    List<String> partitions = Lists.newArrayList();
+    List<String> partitions = new ArrayList<>();
 
     // generates partition column names: dir0, dir1 etc.
     for (int i = 0; i < partitionsCount; i++) {
@@ -165,7 +166,7 @@ public class ColumnExplorer {
    * @param includeFileImplicitColumns if file implicit columns should be included into the result
    * @return implicit columns map
    */
-  public Map<String, String> populateImplicitColumns(String filePath,
+  public Map<String, String> populateImplicitColumns(Path filePath,
                                                      List<String> partitionValues,
                                                      boolean includeFileImplicitColumns) {
     Map<String, String> implicitValues = new LinkedHashMap<>();
@@ -177,7 +178,7 @@ public class ColumnExplorer {
     }
 
     if (includeFileImplicitColumns) {
-      Path path = Path.getPathWithoutSchemeAndAuthority(new Path(filePath));
+      Path path = Path.getPathWithoutSchemeAndAuthority(filePath);
       for (Map.Entry<String, ImplicitFileColumns> entry : selectedImplicitColumns.entrySet()) {
         implicitValues.put(entry.getKey(), entry.getValue().getValue(path));
       }
@@ -189,15 +190,16 @@ public class ColumnExplorer {
   /**
    * Compares root and file path to determine directories
    * that are present in the file path but absent in root.
-   * Example: root - a/b/c, filePath - a/b/c/d/e/0_0_0.parquet, result - d/e.
+   * Example: root - a/b/c, file - a/b/c/d/e/0_0_0.parquet, result - d/e.
    * Stores different directory names in the list in successive order.
    *
-   * @param filePath file path
+   * @param file file path
    * @param root root directory
+   * @param hasDirsOnly whether it is file or directory
    * @return list of directory names
    */
-  public static List<String> listPartitionValues(String filePath, String root) {
-    String[] dirs = parsePartitions(filePath, root);
+  public static List<String> listPartitionValues(Path file, Path root, boolean hasDirsOnly) {
+    String[] dirs = parsePartitions(file, root, hasDirsOnly);
     if (dirs == null) {
       return Collections.emptyList();
     }
@@ -208,21 +210,23 @@ public class ColumnExplorer {
    * Low-level parse of partitions, returned as a string array. Returns a
    * null array for invalid values.
    *
-   * @param filePath file path
+   * @param file file path
    * @param root root directory
+   * @param hasDirsOnly whether it is file or directory
    * @return array of directory names, or null if the arguments are invalid
    */
-  public static String[] parsePartitions(String filePath, String root) {
-    if (filePath == null || root == null) {
+  public static String[] parsePartitions(Path file, Path root, boolean hasDirsOnly) {
+    if (file == null || root == null) {
       return null;
     }
 
-    int rootDepth = new Path(root).depth();
-    Path path = new Path(filePath);
-    int parentDepth = path.getParent().depth();
-
-    int diffCount = parentDepth - rootDepth;
+    if (!hasDirsOnly) {
+      file = file.getParent();
+    }
 
+    int rootDepth = root.depth();
+    int fileDepth = file.depth();
+    int diffCount = fileDepth - rootDepth;
     if (diffCount < 0) {
       return null;
     }
@@ -230,10 +234,10 @@ public class ColumnExplorer {
     String[] diffDirectoryNames = new String[diffCount];
 
     // start filling in array from the end
-    for (int i = rootDepth; parentDepth > i; i++) {
-      path = path.getParent();
+    for (int i = rootDepth; fileDepth > i; i++) {
       // place in the end of array
-      diffDirectoryNames[parentDepth - i - 1] = path.getName();
+      diffDirectoryNames[fileDepth - i - 1] = file.getName();
+      file = file.getParent();
     }
 
     return diffDirectoryNames;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroDrillTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroDrillTable.java
index 50d7ee8..10d90d4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroDrillTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroDrillTable.java
@@ -54,8 +54,8 @@ public class AvroDrillTable extends DrillTable {
                        SchemaConfig schemaConfig,
                        FormatSelection selection) {
     super(storageEngineName, plugin, schemaConfig.getUserName(), selection);
-    List<String> asFiles = selection.getAsFiles();
-    Path path = new Path(asFiles.get(0));
+    List<Path> asFiles = selection.getAsFiles();
+    Path path = asFiles.get(0);
     this.schemaConfig = schemaConfig;
     try {
       reader = new DataFileReader<>(new FsInput(path, plugin.getFsConf()), new GenericDatumReader<GenericContainer>());
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroRecordReader.java
index 1d7226a..07444e9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroRecordReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroRecordReader.java
@@ -90,13 +90,13 @@ public class AvroRecordReader extends AbstractRecordReader {
 
 
   public AvroRecordReader(final FragmentContext fragmentContext,
-                          final String inputPath,
+                          final Path inputPath,
                           final long start,
                           final long length,
                           final FileSystem fileSystem,
                           final List<SchemaPath> projectedColumns,
                           final String userName) {
-    hadoop = new Path(inputPath);
+    hadoop = inputPath;
     this.start = start;
     this.end = start + length;
     buffer = fragmentContext.getManagedBuffer();
@@ -111,12 +111,8 @@ public class AvroRecordReader extends AbstractRecordReader {
   private DataFileReader<GenericContainer> getReader(final Path hadoop, final FileSystem fs) throws ExecutionSetupException {
     try {
       final UserGroupInformation ugi = ImpersonationUtil.createProxyUgi(this.opUserName, this.queryUserName);
-      return ugi.doAs(new PrivilegedExceptionAction<DataFileReader<GenericContainer>>() {
-        @Override
-        public DataFileReader<GenericContainer> run() throws Exception {
-          return new DataFileReader<>(new FsInput(hadoop, fs.getConf()), new GenericDatumReader<GenericContainer>());
-        }
-      });
+      return ugi.doAs((PrivilegedExceptionAction<DataFileReader<GenericContainer>>) () ->
+              new DataFileReader<>(new FsInput(hadoop, fs.getConf()), new GenericDatumReader<GenericContainer>()));
     } catch (IOException | InterruptedException e) {
       throw new ExecutionSetupException(
         String.format("Error in creating avro reader for file: %s", hadoop), e);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSelection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSelection.java
index 2fa9558..902abda 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSelection.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSelection.java
@@ -19,12 +19,15 @@ package org.apache.drill.exec.store.dfs;
 
 import java.io.IOException;
 import java.net.URI;
+import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
+import java.util.Objects;
 import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
 import org.apache.drill.shaded.guava.com.google.common.base.Stopwatch;
-import org.apache.drill.shaded.guava.com.google.common.base.Strings;
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
 
 import org.apache.drill.common.exceptions.DrillRuntimeException;
@@ -41,15 +44,15 @@ public class FileSelection {
 
   private List<FileStatus> statuses;
 
-  public List<String> files;
+  public List<Path> files;
   /**
    * root path for the selections
    */
-  public final String selectionRoot;
+  public final Path selectionRoot;
   /**
    * root path for the metadata cache file (if any)
    */
-  public final String cacheFileRoot;
+  public final Path cacheFileRoot;
 
   /**
    * metadata context useful for metadata operations (if any)
@@ -82,17 +85,17 @@ public class FileSelection {
    * @param files  list of files
    * @param selectionRoot  root path for selections
    */
-  public FileSelection(final List<FileStatus> statuses, final List<String> files, final String selectionRoot) {
+  public FileSelection(List<FileStatus> statuses, List<Path> files, Path selectionRoot) {
     this(statuses, files, selectionRoot, null, false, StatusType.NOT_CHECKED);
   }
 
-  public FileSelection(final List<FileStatus> statuses, final List<String> files, final String selectionRoot,
-      final String cacheFileRoot, final boolean wasAllPartitionsPruned) {
+  public FileSelection(List<FileStatus> statuses, List<Path> files, Path selectionRoot, Path cacheFileRoot,
+      boolean wasAllPartitionsPruned) {
     this(statuses, files, selectionRoot, cacheFileRoot, wasAllPartitionsPruned, StatusType.NOT_CHECKED);
   }
 
-  public FileSelection(final List<FileStatus> statuses, final List<String> files, final String selectionRoot,
-      final String cacheFileRoot, final boolean wasAllPartitionsPruned, final StatusType dirStatus) {
+  public FileSelection(List<FileStatus> statuses, List<Path> files, Path selectionRoot, Path cacheFileRoot,
+      boolean wasAllPartitionsPruned, StatusType dirStatus) {
     this.statuses = statuses;
     this.files = files;
     this.selectionRoot = selectionRoot;
@@ -104,7 +107,7 @@ public class FileSelection {
   /**
    * Copy constructor for convenience.
    */
-  protected FileSelection(final FileSelection selection) {
+  protected FileSelection(FileSelection selection) {
     Preconditions.checkNotNull(selection, "selection cannot be null");
     this.statuses = selection.statuses;
     this.files = selection.files;
@@ -116,17 +119,17 @@ public class FileSelection {
     this.wasAllPartitionsPruned = selection.wasAllPartitionsPruned;
   }
 
-  public String getSelectionRoot() {
+  public Path getSelectionRoot() {
     return selectionRoot;
   }
 
-  public List<FileStatus> getStatuses(final DrillFileSystem fs) throws IOException {
+  public List<FileStatus> getStatuses(DrillFileSystem fs) throws IOException {
     Stopwatch timer = logger.isDebugEnabled() ? Stopwatch.createStarted() : null;
 
     if (statuses == null)  {
-      final List<FileStatus> newStatuses = Lists.newArrayList();
-      for (final String pathStr:files) {
-        newStatuses.add(fs.getFileStatus(new Path(pathStr)));
+      List<FileStatus> newStatuses = Lists.newArrayList();
+      for (Path pathStr : files) {
+        newStatuses.add(fs.getFileStatus(pathStr));
       }
       statuses = newStatuses;
     }
@@ -139,11 +142,11 @@ public class FileSelection {
     return statuses;
   }
 
-  public List<String> getFiles() {
+  public List<Path> getFiles() {
     if (files == null) {
-      final List<String> newFiles = Lists.newArrayList();
-      for (final FileStatus status:statuses) {
-        newFiles.add(status.getPath().toString());
+      List<Path> newFiles = Lists.newArrayList();
+      for (FileStatus status:statuses) {
+        newFiles.add(status.getPath());
       }
       files = newFiles;
     }
@@ -153,7 +156,7 @@ public class FileSelection {
   public boolean containsDirectories(DrillFileSystem fs) throws IOException {
     if (dirStatus == StatusType.NOT_CHECKED) {
       dirStatus = StatusType.NO_DIRS;
-      for (final FileStatus status : getStatuses(fs)) {
+      for (FileStatus status : getStatuses(fs)) {
         if (status.isDirectory()) {
           dirStatus = StatusType.HAS_DIRS;
           break;
@@ -175,7 +178,7 @@ public class FileSelection {
       nonDirectories.addAll(DrillFileSystemUtil.listFiles(fs, status.getPath(), true));
     }
 
-    final FileSelection fileSel = create(nonDirectories, null, selectionRoot);
+    FileSelection fileSel = create(nonDirectories, null, selectionRoot);
     if (timer != null) {
       logger.debug("FileSelection.minusDirectories() took {} ms, numFiles: {}", timer.elapsed(TimeUnit.MILLISECONDS), statuses.size());
       timer.stop();
@@ -223,38 +226,36 @@ public class FileSelection {
    * @param files  list of files.
    * @return  longest common path
    */
-  private static String commonPathForFiles(final List<String> files) {
+  private static Path commonPathForFiles(List<Path> files) {
     if (files == null || files.isEmpty()) {
-      return "";
+      return new Path("/");
     }
 
-    final int total = files.size();
-    final String[][] folders = new String[total][];
+    int total = files.size();
+    String[][] folders = new String[total][];
     int shortest = Integer.MAX_VALUE;
     for (int i = 0; i < total; i++) {
-      final Path path = new Path(files.get(i));
-      folders[i] = Path.getPathWithoutSchemeAndAuthority(path).toString().split(Path.SEPARATOR);
+      folders[i] = files.get(i).toUri().getPath().split(Path.SEPARATOR);
       shortest = Math.min(shortest, folders[i].length);
     }
 
     int latest;
     out:
     for (latest = 0; latest < shortest; latest++) {
-      final String current = folders[0][latest];
+      String current = folders[0][latest];
       for (int i = 1; i < folders.length; i++) {
         if (!current.equals(folders[i][latest])) {
           break out;
         }
       }
     }
-    final Path path = new Path(files.get(0));
-    final URI uri = path.toUri();
-    final String pathString = buildPath(folders[0], latest);
-    return new Path(uri.getScheme(), uri.getAuthority(), pathString).toString();
+    URI uri = files.get(0).toUri();
+    String pathString = buildPath(folders[0], latest);
+    return new Path(uri.getScheme(), uri.getAuthority(), pathString);
   }
 
-  private static String buildPath(final String[] path, final int folderIndex) {
-    final StringBuilder builder = new StringBuilder();
+  private static String buildPath(String[] path, int folderIndex) {
+    StringBuilder builder = new StringBuilder();
     for (int i=0; i<folderIndex; i++) {
       builder.append(path[i]).append(Path.SEPARATOR);
     }
@@ -262,20 +263,20 @@ public class FileSelection {
     return builder.toString();
   }
 
-  public static FileSelection create(final DrillFileSystem fs, final String parent, final String path,
-      final boolean allowAccessOutsideWorkspace) throws IOException {
+  public static FileSelection create(DrillFileSystem fs, String parent, String path,
+      boolean allowAccessOutsideWorkspace) throws IOException {
     Stopwatch timer = logger.isDebugEnabled() ? Stopwatch.createStarted() : null;
     boolean hasWildcard = path.contains(WILD_CARD);
 
-    final Path combined = new Path(parent, removeLeadingSlash(path));
+    Path combined = new Path(parent, removeLeadingSlash(path));
     if (!allowAccessOutsideWorkspace) {
       checkBackPaths(new Path(parent).toUri().getPath(), combined.toUri().getPath(), path);
     }
-    final FileStatus[] statuses = fs.globStatus(combined); // note: this would expand wildcards
+    FileStatus[] statuses = fs.globStatus(combined); // note: this would expand wildcards
     if (statuses == null) {
       return null;
     }
-    final FileSelection fileSel = create(Lists.newArrayList(statuses), null, combined.toUri().getPath());
+    FileSelection fileSel = create(Arrays.asList(statuses), null, combined);
     if (timer != null) {
       logger.debug("FileSelection.create() took {} ms ", timer.elapsed(TimeUnit.MILLISECONDS));
       timer.stop();
@@ -298,62 +299,51 @@ public class FileSelection {
    * @return  null if creation of {@link FileSelection} fails with an {@link IllegalArgumentException}
    *          otherwise a new selection.
    *
-   * @see FileSelection#FileSelection(List, List, String)
+   * @see FileSelection#FileSelection(List, List, Path)
    */
-  public static FileSelection create(final List<FileStatus> statuses, final List<String> files, final String root,
-      final String cacheFileRoot, final boolean wasAllPartitionsPruned) {
-    final boolean bothNonEmptySelection = (statuses != null && statuses.size() > 0) && (files != null && files.size() > 0);
-    final boolean bothEmptySelection = (statuses == null || statuses.size() == 0) && (files == null || files.size() == 0);
+  public static FileSelection create(List<FileStatus> statuses, List<Path> files, Path root,
+      Path cacheFileRoot, boolean wasAllPartitionsPruned) {
+    boolean bothNonEmptySelection = (statuses != null && statuses.size() > 0) && (files != null && files.size() > 0);
+    boolean bothEmptySelection = (statuses == null || statuses.size() == 0) && (files == null || files.size() == 0);
 
     if (bothNonEmptySelection || bothEmptySelection) {
       return null;
     }
 
-    final String selectionRoot;
+    Path selectionRoot;
     if (statuses == null || statuses.isEmpty()) {
       selectionRoot = commonPathForFiles(files);
     } else {
-      if (Strings.isNullOrEmpty(root)) {
-        throw new DrillRuntimeException("Selection root is null or empty" + root);
-      }
-      final Path rootPath = handleWildCard(root);
-      final URI uri = statuses.get(0).getPath().toUri();
-      final Path path = new Path(uri.getScheme(), uri.getAuthority(), rootPath.toUri().getPath());
-      selectionRoot = path.toString();
+      Objects.requireNonNull(root, "Selection root is null");
+      Path rootPath = handleWildCard(root);
+      URI uri = statuses.get(0).getPath().toUri();
+      selectionRoot = new Path(uri.getScheme(), uri.getAuthority(), rootPath.toUri().getPath());
     }
     return new FileSelection(statuses, files, selectionRoot, cacheFileRoot, wasAllPartitionsPruned);
   }
 
-  public static FileSelection create(final List<FileStatus> statuses, final List<String> files, final String root) {
+  public static FileSelection create(List<FileStatus> statuses, List<Path> files, Path root) {
     return FileSelection.create(statuses, files, root, null, false);
   }
 
-  public static FileSelection createFromDirectories(final List<String> dirPaths, final FileSelection selection,
-      final String cacheFileRoot) {
+  public static FileSelection createFromDirectories(List<Path> dirPaths, FileSelection selection,
+      Path cacheFileRoot) {
     Stopwatch timer = logger.isDebugEnabled() ? Stopwatch.createStarted() : null;
-    final String root = selection.getSelectionRoot();
-    if (Strings.isNullOrEmpty(root)) {
-      throw new DrillRuntimeException("Selection root is null or empty" + root);
-    }
+    Path root = selection.getSelectionRoot();
+    Objects.requireNonNull(root, "Selection root is null");
     if (dirPaths == null || dirPaths.isEmpty()) {
       throw new DrillRuntimeException("List of directories is null or empty");
     }
 
-    List<String> dirs = Lists.newArrayList();
+    // for wildcard the directory list should have already been expanded
+    List<Path> dirs = selection.hadWildcard() ? selection.getFileStatuses().stream()
+        .map(FileStatus::getPath)
+        .collect(Collectors.toList()) : new ArrayList<>(dirPaths);
 
-    if (selection.hadWildcard()) { // for wildcard the directory list should have already been expanded
-      for (FileStatus status : selection.getFileStatuses()) {
-        dirs.add(status.getPath().toString());
-      }
-    } else {
-      dirs.addAll(dirPaths);
-    }
-
-    final Path rootPath = handleWildCard(root);
-    // final URI uri = dirPaths.get(0).toUri();
-    final URI uri = selection.getFileStatuses().get(0).getPath().toUri();
-    final Path path = new Path(uri.getScheme(), uri.getAuthority(), rootPath.toUri().getPath());
-    FileSelection fileSel = new FileSelection(null, dirs, path.toString(), cacheFileRoot, false);
+    Path rootPath = handleWildCard(root);
+    URI uri = selection.getFileStatuses().get(0).getPath().toUri();
+    Path path = new Path(uri.getScheme(), uri.getAuthority(), rootPath.toUri().getPath());
+    FileSelection fileSel = new FileSelection(null, dirs, path, cacheFileRoot, false);
     fileSel.setHadWildcard(selection.hadWildcard());
     if (timer != null) {
       logger.debug("FileSelection.createFromDirectories() took {} ms ", timer.elapsed(TimeUnit.MILLISECONDS));
@@ -362,18 +352,15 @@ public class FileSelection {
     return fileSel;
   }
 
-  private static Path handleWildCard(final String root) {
-    if (root.contains(WILD_CARD)) {
-      int idx = root.indexOf(WILD_CARD); // first wild card in the path
-      idx = root.lastIndexOf('/', idx); // file separator right before the first wild card
-      final String newRoot = root.substring(0, idx);
-      if (newRoot.length() == 0) {
-          // Ensure that we always return a valid root.
-          return new Path("/");
-      }
-      return new Path(newRoot);
+  private static Path handleWildCard(Path root) {
+    String stringRoot = root.toUri().getPath();
+    if (stringRoot.contains(WILD_CARD)) {
+      int idx = stringRoot.indexOf(WILD_CARD); // first wild card in the path
+      idx = stringRoot.lastIndexOf('/', idx); // file separator right before the first wild card
+      String newRoot = stringRoot.substring(0, idx);
+      return DrillFileSystemUtil.createPathSafe(newRoot);
     } else {
-      return new Path(root);
+      return new Path(stringRoot);
     }
   }
 
@@ -426,7 +413,7 @@ public class FileSelection {
     return this.hadWildcard;
   }
 
-  public String getCacheFileRoot() {
+  public Path getCacheFileRoot() {
     return cacheFileRoot;
   }
 
@@ -456,12 +443,12 @@ public class FileSelection {
 
   @Override
   public String toString() {
-    final StringBuilder sb = new StringBuilder();
+    StringBuilder sb = new StringBuilder();
     sb.append("root=").append(this.selectionRoot);
 
     sb.append("files=[");
     boolean isFirst = true;
-    for (final String file : this.files) {
+    for (Path file : this.files) {
       if (isFirst) {
         isFirst = false;
         sb.append(file);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FormatSelection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FormatSelection.java
index 40549cc..7d7bcfa 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FormatSelection.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FormatSelection.java
@@ -24,6 +24,7 @@ import org.apache.drill.common.logical.FormatPluginConfig;
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.hadoop.fs.Path;
 
 
 public class FormatSelection {
@@ -32,16 +33,15 @@ public class FormatSelection {
   private FormatPluginConfig format;
   private FileSelection selection;
 
-  public FormatSelection(){}
+  public FormatSelection() {}
 
   @JsonCreator
-  public FormatSelection(@JsonProperty("format") FormatPluginConfig format, @JsonProperty("files") List<String> files){
+  public FormatSelection(@JsonProperty("format") FormatPluginConfig format, @JsonProperty("files") List<Path> files){
     this.format = format;
     this.selection = FileSelection.create(null, files, null);
   }
 
   public FormatSelection(FormatPluginConfig format, FileSelection selection) {
-    super();
     this.format = format;
     this.selection = selection;
   }
@@ -52,7 +52,7 @@ public class FormatSelection {
   }
 
   @JsonProperty("files")
-  public List<String> getAsFiles(){
+  public List<Path> getAsFiles(){
     return selection.getFiles();
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/MetadataContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/MetadataContext.java
index 877ceb6..0738478 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/MetadataContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/MetadataContext.java
@@ -20,6 +20,7 @@ package org.apache.drill.exec.store.dfs;
 import java.util.Map;
 
 import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
+import org.apache.hadoop.fs.Path;
 
 /**
  * A metadata context that holds state across multiple invocations of
@@ -32,17 +33,17 @@ public class MetadataContext {
    *  Note: the #directories is typically a small percentage of the #files, so the memory footprint
    *  is expected to be relatively small.
    */
-  private Map<String, Boolean> dirModifCheckMap = Maps.newHashMap();
+  private Map<Path, Boolean> dirModifCheckMap = Maps.newHashMap();
 
   private PruneStatus pruneStatus = PruneStatus.NOT_STARTED;
 
   private boolean metadataCacheCorrupted;
 
-  public void setStatus(String dir) {
+  public void setStatus(Path dir) {
     dirModifCheckMap.put(dir,  true);
   }
 
-  public void clearStatus(String dir) {
+  public void clearStatus(Path dir) {
     dirModifCheckMap.put(dir,  false);
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/ReadEntryFromHDFS.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/ReadEntryFromHDFS.java
index 15107ac..ec925e3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/ReadEntryFromHDFS.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/ReadEntryFromHDFS.java
@@ -21,6 +21,7 @@ import org.apache.drill.exec.store.dfs.easy.FileWork;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.hadoop.fs.Path;
 
 public class ReadEntryFromHDFS extends ReadEntryWithPath implements FileWork {
 
@@ -28,7 +29,7 @@ public class ReadEntryFromHDFS extends ReadEntryWithPath implements FileWork {
   private long length;
 
   @JsonCreator
-  public ReadEntryFromHDFS(@JsonProperty("path") String path,@JsonProperty("start") long start, @JsonProperty("length") long length) {
+  public ReadEntryFromHDFS(@JsonProperty("path") Path path, @JsonProperty("start") long start, @JsonProperty("length") long length) {
     super(path);
     this.start = start;
     this.length = length;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/ReadEntryWithPath.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/ReadEntryWithPath.java
index 4564e15..88bd9fb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/ReadEntryWithPath.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/ReadEntryWithPath.java
@@ -18,19 +18,20 @@
 package org.apache.drill.exec.store.dfs;
 
 
+import org.apache.hadoop.fs.Path;
+
 public class ReadEntryWithPath {
 
-  protected String path;
+  protected Path path;
 
+  // Default constructor is needed for deserialization
+  public ReadEntryWithPath() {}
 
-  public ReadEntryWithPath(String path) {
-    super();
+  public ReadEntryWithPath(Path path) {
     this.path = path;
   }
 
-  public ReadEntryWithPath(){}
-
-  public String getPath(){
+  public Path getPath() {
    return path;
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
index d3bed8f..d76c648 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
@@ -156,7 +156,7 @@ public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements
     for (FileWork work : scan.getWorkUnits()){
       RecordReader recordReader = getRecordReader(context, dfs, work, scan.getColumns(), scan.getUserName());
       readers.add(recordReader);
-      List<String> partitionValues = ColumnExplorer.listPartitionValues(work.getPath(), scan.getSelectionRoot());
+      List<String> partitionValues = ColumnExplorer.listPartitionValues(work.getPath(), scan.getSelectionRoot(), false);
       Map<String, String> implicitValues = columnExplorer.populateImplicitColumns(work.getPath(), partitionValues, supportsFileImplicitColumns);
       implicitColumns.add(implicitValues);
       if (implicitValues.size() > mapWithMaxColumns.size()) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
index 759d07f..4449ec0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
@@ -18,7 +18,6 @@
 package org.apache.drill.exec.store.dfs.easy;
 
 import java.io.IOException;
-import java.util.Iterator;
 import java.util.List;
 
 import org.apache.drill.common.exceptions.ExecutionSetupException;
@@ -52,6 +51,7 @@ import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
 import org.apache.drill.shaded.guava.com.google.common.collect.Iterators;
 import org.apache.drill.shaded.guava.com.google.common.collect.ListMultimap;
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.hadoop.fs.Path;
 
 @JsonTypeName("fs-scan")
 public class EasyGroupScan extends AbstractFileGroupScan {
@@ -65,17 +65,17 @@ public class EasyGroupScan extends AbstractFileGroupScan {
   private ListMultimap<Integer, CompleteFileWork> mappings;
   private List<CompleteFileWork> chunks;
   private List<EndpointAffinity> endpointAffinities;
-  private String selectionRoot;
+  private Path selectionRoot;
 
   @JsonCreator
   public EasyGroupScan(
       @JsonProperty("userName") String userName,
-      @JsonProperty("files") List<String> files, //
-      @JsonProperty("storage") StoragePluginConfig storageConfig, //
-      @JsonProperty("format") FormatPluginConfig formatConfig, //
-      @JacksonInject StoragePluginRegistry engineRegistry, //
+      @JsonProperty("files") List<Path> files,
+      @JsonProperty("storage") StoragePluginConfig storageConfig,
+      @JsonProperty("format") FormatPluginConfig formatConfig,
+      @JacksonInject StoragePluginRegistry engineRegistry,
       @JsonProperty("columns") List<SchemaPath> columns,
-      @JsonProperty("selectionRoot") String selectionRoot
+      @JsonProperty("selectionRoot") Path selectionRoot
       ) throws IOException, ExecutionSetupException {
         this(ImpersonationUtil.resolveUserName(userName),
             FileSelection.create(null, files, selectionRoot),
@@ -84,17 +84,17 @@ public class EasyGroupScan extends AbstractFileGroupScan {
             selectionRoot);
   }
 
-  public EasyGroupScan(String userName, FileSelection selection, EasyFormatPlugin<?> formatPlugin, String selectionRoot)
+  public EasyGroupScan(String userName, FileSelection selection, EasyFormatPlugin<?> formatPlugin, Path selectionRoot)
       throws IOException {
     this(userName, selection, formatPlugin, ALL_COLUMNS, selectionRoot);
   }
 
   public EasyGroupScan(
       String userName,
-      FileSelection selection, //
-      EasyFormatPlugin<?> formatPlugin, //
+      FileSelection selection,
+      EasyFormatPlugin<?> formatPlugin,
       List<SchemaPath> columns,
-      String selectionRoot
+      Path selectionRoot
       ) throws IOException{
     super(userName);
     this.selection = Preconditions.checkNotNull(selection);
@@ -106,12 +106,7 @@ public class EasyGroupScan extends AbstractFileGroupScan {
 
   @JsonIgnore
   public Iterable<CompleteFileWork> getWorkIterable() {
-    return new Iterable<CompleteFileWork>() {
-      @Override
-      public Iterator<CompleteFileWork> iterator() {
-        return Iterators.unmodifiableIterator(chunks.iterator());
-      }
-    };
+    return () -> Iterators.unmodifiableIterator(chunks.iterator());
   }
 
   private EasyGroupScan(final EasyGroupScan that) {
@@ -136,7 +131,7 @@ public class EasyGroupScan extends AbstractFileGroupScan {
     this.endpointAffinities = AffinityCreator.getAffinityMap(chunks);
   }
 
-  public String getSelectionRoot() {
+  public Path getSelectionRoot() {
     return selectionRoot;
   }
 
@@ -158,7 +153,7 @@ public class EasyGroupScan extends AbstractFileGroupScan {
 
   @JsonProperty("files")
   @Override
-  public List<String> getFiles() {
+  public List<Path> getFiles() {
     return selection.getFiles();
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasySubScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasySubScan.java
index 0dbae1e..fbb3f47 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasySubScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasySubScan.java
@@ -17,7 +17,6 @@
  */
 package org.apache.drill.exec.store.dfs.easy;
 
-import java.io.IOException;
 import java.util.List;
 
 import org.apache.drill.common.exceptions.ExecutionSetupException;
@@ -34,6 +33,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+import org.apache.hadoop.fs.Path;
 
 @JsonTypeName("fs-sub-scan")
 public class EasySubScan extends AbstractSubScan{
@@ -42,18 +42,18 @@ public class EasySubScan extends AbstractSubScan{
   private final List<FileWorkImpl> files;
   private final EasyFormatPlugin<?> formatPlugin;
   private final List<SchemaPath> columns;
-  private String selectionRoot;
+  private Path selectionRoot;
 
   @JsonCreator
   public EasySubScan(
       @JsonProperty("userName") String userName,
-      @JsonProperty("files") List<FileWorkImpl> files, //
-      @JsonProperty("storage") StoragePluginConfig storageConfig, //
-      @JsonProperty("format") FormatPluginConfig formatConfig, //
-      @JacksonInject StoragePluginRegistry engineRegistry, //
-      @JsonProperty("columns") List<SchemaPath> columns, //
-      @JsonProperty("selectionRoot") String selectionRoot
-      ) throws IOException, ExecutionSetupException {
+      @JsonProperty("files") List<FileWorkImpl> files,
+      @JsonProperty("storage") StoragePluginConfig storageConfig,
+      @JsonProperty("format") FormatPluginConfig formatConfig,
+      @JacksonInject StoragePluginRegistry engineRegistry,
+      @JsonProperty("columns") List<SchemaPath> columns,
+      @JsonProperty("selectionRoot") Path selectionRoot
+      ) throws ExecutionSetupException {
     super(userName);
     this.formatPlugin = (EasyFormatPlugin<?>) engineRegistry.getFormatPlugin(storageConfig, formatConfig);
     Preconditions.checkNotNull(this.formatPlugin);
@@ -63,7 +63,7 @@ public class EasySubScan extends AbstractSubScan{
   }
 
   public EasySubScan(String userName, List<FileWorkImpl> files, EasyFormatPlugin<?> plugin, List<SchemaPath> columns,
-      String selectionRoot){
+      Path selectionRoot){
     super(userName);
     this.formatPlugin = plugin;
     this.files = files;
@@ -72,7 +72,7 @@ public class EasySubScan extends AbstractSubScan{
   }
 
   @JsonProperty
-  public String getSelectionRoot() {
+  public Path getSelectionRoot() {
     return selectionRoot;
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/FileWork.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/FileWork.java
index 587201e..3aeb2c2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/FileWork.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/FileWork.java
@@ -17,8 +17,14 @@
  */
 package org.apache.drill.exec.store.dfs.easy;
 
+
+import org.apache.hadoop.fs.Path;
+
 public interface FileWork {
-  String getPath();
+
+  Path getPath();
+
   long getStart();
+
   long getLength();
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/MetadataDirectGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/MetadataDirectGroupScan.java
index 505d68e..6de8842 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/MetadataDirectGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/MetadataDirectGroupScan.java
@@ -18,12 +18,12 @@
 package org.apache.drill.exec.store.direct;
 
 import com.fasterxml.jackson.annotation.JsonTypeName;
-import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.physical.base.GroupScan;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.ScanStats;
 import org.apache.drill.exec.store.RecordReader;
+import org.apache.hadoop.fs.Path;
 
 import java.util.Collection;
 import java.util.List;
@@ -37,20 +37,20 @@ import java.util.List;
 @JsonTypeName("metadata-direct-scan")
 public class MetadataDirectGroupScan extends DirectGroupScan {
 
-  private final Collection<String> files;
+  private final Collection<Path> files;
 
-  public MetadataDirectGroupScan(RecordReader reader, Collection<String> files) {
+  public MetadataDirectGroupScan(RecordReader reader, Collection<Path> files) {
     super(reader);
     this.files = files;
   }
 
-  public MetadataDirectGroupScan(RecordReader reader, Collection<String> files, ScanStats stats) {
+  public MetadataDirectGroupScan(RecordReader reader, Collection<Path> files, ScanStats stats) {
     super(reader, stats);
     this.files = files;
   }
 
   @Override
-  public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) throws ExecutionSetupException {
+  public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
     assert children == null || children.isEmpty();
     return new MetadataDirectGroupScan(reader, files, stats);
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONRecordReader.java
index 428a4e1..d3fcc5a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONRecordReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONRecordReader.java
@@ -77,7 +77,7 @@ public class JSONRecordReader extends AbstractRecordReader {
    * @param columns  pathnames of columns/subfields to read
    * @throws OutOfMemoryException
    */
-  public JSONRecordReader(final FragmentContext fragmentContext, final String inputPath, final DrillFileSystem fileSystem,
+  public JSONRecordReader(final FragmentContext fragmentContext, final Path inputPath, final DrillFileSystem fileSystem,
       final List<SchemaPath> columns) throws OutOfMemoryException {
     this(fragmentContext, inputPath, null, fileSystem, columns);
   }
@@ -90,14 +90,13 @@ public class JSONRecordReader extends AbstractRecordReader {
    * @param columns  pathnames of columns/subfields to read
    * @throws OutOfMemoryException
    */
-  public JSONRecordReader(final FragmentContext fragmentContext, final JsonNode embeddedContent,
-      final DrillFileSystem fileSystem, final List<SchemaPath> columns) throws OutOfMemoryException {
+  public JSONRecordReader(FragmentContext fragmentContext, JsonNode embeddedContent, DrillFileSystem fileSystem,
+      List<SchemaPath> columns) throws OutOfMemoryException {
     this(fragmentContext, null, embeddedContent, fileSystem, columns);
   }
 
-  private JSONRecordReader(final FragmentContext fragmentContext, final String inputPath,
-      final JsonNode embeddedContent, final DrillFileSystem fileSystem,
-      final List<SchemaPath> columns) {
+  private JSONRecordReader(FragmentContext fragmentContext, Path inputPath, JsonNode embeddedContent,
+      DrillFileSystem fileSystem, List<SchemaPath> columns) {
 
     Preconditions.checkArgument(
         (inputPath == null && embeddedContent != null) ||
@@ -106,7 +105,7 @@ public class JSONRecordReader extends AbstractRecordReader {
         );
 
     if (inputPath != null) {
-      this.hadoopPath = new Path(inputPath);
+      this.hadoopPath = inputPath;
     } else {
       this.embeddedContent = embeddedContent;
     }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/sequencefile/SequenceFileFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/sequencefile/SequenceFileFormatPlugin.java
index 9dbe715..ec4bb12 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/sequencefile/SequenceFileFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/sequencefile/SequenceFileFormatPlugin.java
@@ -87,7 +87,7 @@ public class SequenceFileFormatPlugin extends EasyFormatPlugin<SequenceFileForma
                                       FileWork fileWork,
                                       List<SchemaPath> columns,
                                       String userName) throws ExecutionSetupException {
-    final Path path = dfs.makeQualified(new Path(fileWork.getPath()));
+    final Path path = dfs.makeQualified(fileWork.getPath());
     final FileSplit split = new FileSplit(path, fileWork.getStart(), fileWork.getLength(), new String[]{""});
     return new SequenceFileRecordReader(split, dfs, context.getQueryUserName(), userName);
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/TextFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/TextFormatPlugin.java
index 1c53a37..03ae6f5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/TextFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/TextFormatPlugin.java
@@ -82,7 +82,7 @@ public class TextFormatPlugin extends EasyFormatPlugin<TextFormatPlugin.TextForm
                                       FileWork fileWork,
                                       List<SchemaPath> columns,
                                       String userName) {
-    Path path = dfs.makeQualified(new Path(fileWork.getPath()));
+    Path path = dfs.makeQualified(fileWork.getPath());
     FileSplit split = new FileSplit(path, fileWork.getStart(), fileWork.getLength(), new String[]{""});
 
     if (context.getOptions().getBoolean(ExecConstants.ENABLE_NEW_TEXT_READER_KEY)) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogFormatPlugin.java
index 3958a32..5a78732 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogFormatPlugin.java
@@ -192,7 +192,7 @@ public class HttpdLogFormatPlugin extends EasyFormatPlugin<HttpdLogFormatPlugin.
             HttpdLogFormatPlugin.this.getConfig().getTimestampFormat(),
             fieldMapping);
 
-        final Path path = fs.makeQualified(new Path(work.getPath()));
+        final Path path = fs.makeQualified(work.getPath());
         FileSplit split = new FileSplit(path, work.getStart(), work.getLength(), new String[]{""});
         TextInputFormat inputFormat = new TextInputFormat();
         JobConf job = new JobConf(fs.getConf());
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/image/ImageFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/image/ImageFormatPlugin.java
index 15ea1b4..048aa82 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/image/ImageFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/image/ImageFormatPlugin.java
@@ -22,7 +22,6 @@ import java.io.IOException;
 import java.util.Collections;
 import java.util.List;
 
-import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.exec.ops.FragmentContext;
@@ -57,11 +56,9 @@ public class ImageFormatPlugin extends EasyFormatPlugin<ImageFormatConfig> {
 
   @Override
   public RecordReader getRecordReader(FragmentContext context, DrillFileSystem dfs, FileWork fileWork,
-      List<SchemaPath> columns, String userName) throws ExecutionSetupException {
-    return new ImageRecordReader(context, dfs, fileWork.getPath(),
-        ((ImageFormatConfig)formatConfig).hasFileSystemMetadata(),
-        ((ImageFormatConfig)formatConfig).isDescriptive(),
-        ((ImageFormatConfig)formatConfig).getTimeZone());
+      List<SchemaPath> columns, String userName) {
+    return new ImageRecordReader(context, dfs, fileWork.getPath(), formatConfig.hasFileSystemMetadata(),
+        formatConfig.isDescriptive(), formatConfig.getTimeZone());
   }
 
   @Override
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/image/ImageRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/image/ImageRecordReader.java
index 2a4b4fb..08ed4fd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/image/ImageRecordReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/image/ImageRecordReader.java
@@ -99,10 +99,10 @@ public class ImageRecordReader extends AbstractRecordReader {
   private DrillBuf managedBuffer;
   private boolean finish;
 
-  public ImageRecordReader(FragmentContext context, DrillFileSystem fs, String inputPath,
+  public ImageRecordReader(FragmentContext context, DrillFileSystem fs, Path inputPath,
                            boolean fileSystemMetadata, boolean descriptive, String timeZone) {
     this.fs = fs;
-    hadoopPath = fs.makeQualified(new Path(inputPath));
+    hadoopPath = fs.makeQualified(inputPath);
     this.fileSystemMetadata = fileSystemMetadata;
     this.descriptive = descriptive;
     this.timeZone = (timeZone != null) ? TimeZone.getTimeZone(timeZone) : TimeZone.getDefault();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/log/LogRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/log/LogRecordReader.java
index e5d1dc4..ae0b733 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/log/LogRecordReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/log/LogRecordReader.java
@@ -41,9 +41,6 @@ import org.apache.drill.exec.vector.NullableVarCharVector;
 import org.apache.drill.exec.vector.NullableTimeStampVector;
 import org.apache.drill.exec.vector.NullableTimeVector;
 
-
-import org.apache.hadoop.fs.Path;
-
 import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.InputStream;
@@ -522,7 +519,7 @@ public class LogRecordReader extends AbstractRecordReader {
   private void openFile() {
     InputStream in;
     try {
-      in = dfs.open(new Path(fileWork.getPath()));
+      in = dfs.open(fileWork.getPath());
     } catch (Exception e) {
       throw UserException
           .dataReadError(e)
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetGroupScan.java
index e2d3569..5528d02 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetGroupScan.java
@@ -52,6 +52,7 @@ import org.apache.drill.exec.store.schedule.AffinityCreator;
 import org.apache.drill.exec.store.schedule.AssignmentCreator;
 import org.apache.drill.exec.store.schedule.EndpointByteMap;
 import org.apache.drill.exec.store.schedule.EndpointByteMapImpl;
+import org.apache.hadoop.fs.Path;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -80,7 +81,7 @@ public abstract class AbstractParquetGroupScan extends AbstractFileGroupScan {
   protected ParquetTableMetadataBase parquetTableMetadata;
   protected List<RowGroupInfo> rowGroupInfos;
   protected ListMultimap<Integer, RowGroupInfo> mappings;
-  protected Set<String> fileSet;
+  protected Set<Path> fileSet;
   protected ParquetReaderConfig readerConfig;
 
   private List<EndpointAffinity> endpointAffinities;
@@ -146,7 +147,7 @@ public abstract class AbstractParquetGroupScan extends AbstractFileGroupScan {
 
   @JsonIgnore
   @Override
-  public Collection<String> getFiles() {
+  public Collection<Path> getFiles() {
     return fileSet;
   }
 
@@ -428,12 +429,12 @@ public abstract class AbstractParquetGroupScan extends AbstractFileGroupScan {
   }
 
   @JsonIgnore
-  public <T> T getPartitionValue(String path, SchemaPath column, Class<T> clazz) {
+  public <T> T getPartitionValue(Path path, SchemaPath column, Class<T> clazz) {
     return clazz.cast(parquetGroupScanStatistics.getPartitionValue(path, column));
   }
 
   @JsonIgnore
-  public Set<String> getFileSet() {
+  public Set<Path> getFileSet() {
     return fileSet;
   }
   // partition pruning methods end
@@ -441,7 +442,7 @@ public abstract class AbstractParquetGroupScan extends AbstractFileGroupScan {
   // helper method used for partition pruning and filter push down
   @Override
   public void modifyFileSelection(FileSelection selection) {
-    List<String> files = selection.getFiles();
+    List<Path> files = selection.getFiles();
     fileSet = new HashSet<>(files);
     entries = new ArrayList<>(files.size());
 
@@ -464,7 +465,7 @@ public abstract class AbstractParquetGroupScan extends AbstractFileGroupScan {
     if (fileSet == null) {
       fileSet = new HashSet<>();
       fileSet.addAll(parquetTableMetadata.getFiles().stream()
-          .map((Function<ParquetFileMetadata, String>) ParquetFileMetadata::getPath)
+          .map((Function<ParquetFileMetadata, Path>) ParquetFileMetadata::getPath)
           .collect(Collectors.toSet()));
     }
 
@@ -505,7 +506,7 @@ public abstract class AbstractParquetGroupScan extends AbstractFileGroupScan {
   // abstract methods block start
   protected abstract void initInternal() throws IOException;
   protected abstract Collection<CoordinationProtos.DrillbitEndpoint> getDrillbits();
-  protected abstract AbstractParquetGroupScan cloneWithFileSelection(Collection<String> filePaths) throws IOException;
+  protected abstract AbstractParquetGroupScan cloneWithFileSelection(Collection<Path> filePaths) throws IOException;
   protected abstract boolean supportsFileImplicitColumns();
   protected abstract List<String> getPartitionValues(RowGroupInfo rowGroupInfo);
   // abstract methods block end
@@ -520,7 +521,7 @@ public abstract class AbstractParquetGroupScan extends AbstractFileGroupScan {
    * @return new parquet group scan
    */
   private AbstractParquetGroupScan cloneWithRowGroupInfos(List<RowGroupInfo> rowGroupInfos) throws IOException {
-    Set<String> filePaths = rowGroupInfos.stream()
+    Set<Path> filePaths = rowGroupInfos.stream()
       .map(ReadEntryWithPath::getPath)
       .collect(Collectors.toSet()); // set keeps file names unique
     AbstractParquetGroupScan scan = cloneWithFileSelection(filePaths);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetScanBatchCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetScanBatchCreator.java
index 99161fd..b1819e6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetScanBatchCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetScanBatchCreator.java
@@ -52,7 +52,8 @@ public abstract class AbstractParquetScanBatchCreator {
 
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractParquetScanBatchCreator.class);
 
-  protected ScanBatch getBatch(ExecutorFragmentContext context, AbstractParquetRowGroupScan rowGroupScan, OperatorContext oContext) throws ExecutionSetupException {
+  protected ScanBatch getBatch(ExecutorFragmentContext context, AbstractParquetRowGroupScan rowGroupScan,
+                               OperatorContext oContext) throws ExecutionSetupException {
     final ColumnExplorer columnExplorer = new ColumnExplorer(context.getOptions(), rowGroupScan.getColumns());
 
     if (!columnExplorer.isStarQuery()) {
@@ -63,7 +64,7 @@ public abstract class AbstractParquetScanBatchCreator {
     AbstractDrillFileSystemManager fsManager = getDrillFileSystemCreator(oContext, context.getOptions());
 
     // keep footers in a map to avoid re-reading them
-    Map<String, ParquetMetadata> footers = new HashMap<>();
+    Map<Path, ParquetMetadata> footers = new HashMap<>();
     List<RecordReader> readers = new LinkedList<>();
     List<Map<String, String>> implicitColumns = new ArrayList<>();
     Map<String, String> mapWithMaxColumns = new LinkedHashMap<>();
@@ -150,8 +151,8 @@ public abstract class AbstractParquetScanBatchCreator {
 
   protected abstract AbstractDrillFileSystemManager getDrillFileSystemCreator(OperatorContext operatorContext, OptionManager optionManager);
 
-  private ParquetMetadata readFooter(Configuration conf, String path, ParquetReaderConfig readerConfig) throws IOException {
-    try (ParquetFileReader reader = ParquetFileReader.open(HadoopInputFile.fromPath(new Path(path),
+  private ParquetMetadata readFooter(Configuration conf, Path path, ParquetReaderConfig readerConfig) throws IOException {
+    try (ParquetFileReader reader = ParquetFileReader.open(HadoopInputFile.fromPath(path,
       readerConfig.addCountersToConf(conf)), readerConfig.toReadOptions())) {
       return reader.getFooter();
     }
@@ -168,6 +169,6 @@ public abstract class AbstractParquetScanBatchCreator {
       this.operatorContext = operatorContext;
     }
 
-    protected abstract DrillFileSystem get(Configuration config, String path) throws ExecutionSetupException;
+    protected abstract DrillFileSystem get(Configuration config, Path path) throws ExecutionSetupException;
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
index a1d9f51..cb9e9b9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
@@ -68,8 +68,8 @@ public class ParquetGroupScan extends AbstractParquetGroupScan {
   private final MetadataContext metaContext;
   private boolean usedMetadataCache; // false by default
   // may change when filter push down / partition pruning is applied
-  private String selectionRoot;
-  private String cacheFileRoot;
+  private Path selectionRoot;
+  private Path cacheFileRoot;
 
   @JsonCreator
   public ParquetGroupScan(@JacksonInject StoragePluginRegistry engineRegistry,
@@ -78,8 +78,8 @@ public class ParquetGroupScan extends AbstractParquetGroupScan {
                           @JsonProperty("storage") StoragePluginConfig storageConfig,
                           @JsonProperty("format") FormatPluginConfig formatConfig,
                           @JsonProperty("columns") List<SchemaPath> columns,
-                          @JsonProperty("selectionRoot") String selectionRoot,
-                          @JsonProperty("cacheFileRoot") String cacheFileRoot,
+                          @JsonProperty("selectionRoot") Path selectionRoot,
+                          @JsonProperty("cacheFileRoot") Path cacheFileRoot,
                           @JsonProperty("readerConfig") ParquetReaderConfig readerConfig,
                           @JsonProperty("filter") LogicalExpression filter) throws IOException, ExecutionSetupException {
     super(ImpersonationUtil.resolveUserName(userName), columns, entries, readerConfig, filter);
@@ -127,7 +127,7 @@ public class ParquetGroupScan extends AbstractParquetGroupScan {
         // The fully expanded list is already stored as part of the fileSet
         entries.add(new ReadEntryWithPath(fileSelection.getSelectionRoot()));
       } else {
-        for (String fileName : fileSelection.getFiles()) {
+        for (Path fileName : fileSelection.getFiles()) {
           entries.add(new ReadEntryWithPath(fileName));
         }
       }
@@ -169,12 +169,12 @@ public class ParquetGroupScan extends AbstractParquetGroupScan {
   }
 
   @JsonProperty
-  public String getSelectionRoot() {
+  public Path getSelectionRoot() {
     return selectionRoot;
   }
 
   @JsonProperty
-  public String getCacheFileRoot() {
+  public Path getCacheFileRoot() {
     return cacheFileRoot;
   }
   // getters for serialization / deserialization end
@@ -224,8 +224,8 @@ public class ParquetGroupScan extends AbstractParquetGroupScan {
       // For EXPLAIN, remove the URI prefix from cacheFileRoot.  If cacheFileRoot is null, we
       // would have read the cache file from selectionRoot
       String cacheFileRootString = (cacheFileRoot == null) ?
-          Path.getPathWithoutSchemeAndAuthority(new Path(selectionRoot)).toString() :
-          Path.getPathWithoutSchemeAndAuthority(new Path(cacheFileRoot)).toString();
+          Path.getPathWithoutSchemeAndAuthority(selectionRoot).toString() :
+          Path.getPathWithoutSchemeAndAuthority(cacheFileRoot).toString();
       builder.append(", cacheFileRoot=").append(cacheFileRootString);
     }
 
@@ -241,7 +241,7 @@ public class ParquetGroupScan extends AbstractParquetGroupScan {
     FileSystem processUserFileSystem = ImpersonationUtil.createFileSystem(ImpersonationUtil.getProcessUserName(), fs.getConf());
     Path metaPath = null;
     if (entries.size() == 1 && parquetTableMetadata == null) {
-      Path p = Path.getPathWithoutSchemeAndAuthority(new Path(entries.get(0).getPath()));
+      Path p = Path.getPathWithoutSchemeAndAuthority(entries.get(0).getPath());
       if (fs.isDirectory(p)) {
         // Using the metadata file makes sense when querying a directory; otherwise
         // if querying a single file we can look up the metadata directly from the file
@@ -257,9 +257,9 @@ public class ParquetGroupScan extends AbstractParquetGroupScan {
         parquetTableMetadata = Metadata.getParquetTableMetadata(processUserFileSystem, p.toString(), readerConfig);
       }
     } else {
-      Path p = Path.getPathWithoutSchemeAndAuthority(new Path(selectionRoot));
+      Path p = Path.getPathWithoutSchemeAndAuthority(selectionRoot);
       metaPath = new Path(p, Metadata.METADATA_FILENAME);
-      if (!metaContext.isMetadataCacheCorrupted() && fs.isDirectory(new Path(selectionRoot))
+      if (!metaContext.isMetadataCacheCorrupted() && fs.isDirectory(selectionRoot)
           && fs.exists(metaPath)) {
         if (parquetTableMetadata == null) {
           parquetTableMetadata = Metadata.readBlockMeta(processUserFileSystem, metaPath, metaContext, readerConfig);
@@ -275,7 +275,7 @@ public class ParquetGroupScan extends AbstractParquetGroupScan {
         final List<FileStatus> fileStatuses = new ArrayList<>();
         for (ReadEntryWithPath entry : entries) {
           fileStatuses.addAll(
-              DrillFileSystemUtil.listFiles(fs, Path.getPathWithoutSchemeAndAuthority(new Path(entry.getPath())), true));
+              DrillFileSystemUtil.listFiles(fs, Path.getPathWithoutSchemeAndAuthority(entry.getPath()), true));
         }
 
         Map<FileStatus, FileSystem> statusMap = fileStatuses.stream()
@@ -292,7 +292,7 @@ public class ParquetGroupScan extends AbstractParquetGroupScan {
   }
 
   @Override
-  protected AbstractParquetGroupScan cloneWithFileSelection(Collection<String> filePaths) throws IOException {
+  protected AbstractParquetGroupScan cloneWithFileSelection(Collection<Path> filePaths) throws IOException {
     FileSelection newSelection = new FileSelection(null, new ArrayList<>(filePaths), getSelectionRoot(), cacheFileRoot, false);
     return clone(newSelection);
   }
@@ -309,7 +309,7 @@ public class ParquetGroupScan extends AbstractParquetGroupScan {
 
   @Override
   protected List<String> getPartitionValues(RowGroupInfo rowGroupInfo) {
-    return ColumnExplorer.listPartitionValues(rowGroupInfo.getPath(), selectionRoot);
+    return ColumnExplorer.listPartitionValues(rowGroupInfo.getPath(), selectionRoot, false);
   }
   // overridden protected methods block end
 
@@ -425,7 +425,7 @@ public class ParquetGroupScan extends AbstractParquetGroupScan {
           }
         } else {
           final Path path = Path.getPathWithoutSchemeAndAuthority(cacheFileRoot);
-          fileSet.add(path.toString());
+          fileSet.add(path);
         }
       }
     }
@@ -436,21 +436,21 @@ public class ParquetGroupScan extends AbstractParquetGroupScan {
       return null;
     }
 
-    List<String> fileNames = new ArrayList<>(fileSet);
+    List<Path> fileNames = new ArrayList<>(fileSet);
 
     // when creating the file selection, set the selection root without the URI prefix
     // The reason is that the file names above have been created in the form
     // /a/b/c.parquet and the format of the selection root must match that of the file names
     // otherwise downstream operations such as partition pruning can break.
-    final Path metaRootPath = Path.getPathWithoutSchemeAndAuthority(new Path(selection.getSelectionRoot()));
-    this.selectionRoot = metaRootPath.toString();
+    final Path metaRootPath = Path.getPathWithoutSchemeAndAuthority(selection.getSelectionRoot());
+    this.selectionRoot = metaRootPath;
 
     // Use the FileSelection constructor directly here instead of the FileSelection.create() method
     // because create() changes the root to include the scheme and authority; In future, if create()
     // is the preferred way to instantiate a file selection, we may need to do something different...
     // WARNING: file statuses and file names are inconsistent
-    FileSelection newSelection = new FileSelection(selection.getStatuses(fs), fileNames, metaRootPath.toString(),
-        cacheFileRoot, selection.wasAllPartitionsPruned());
+    FileSelection newSelection = new FileSelection(selection.getStatuses(fs), fileNames, metaRootPath, cacheFileRoot,
+        selection.wasAllPartitionsPruned());
 
     newSelection.setExpandedFully();
     newSelection.setMetaContext(metaContext);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScanStatistics.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScanStatistics.java
index 9381043..9156524 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScanStatistics.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScanStatistics.java
@@ -21,6 +21,7 @@ import org.apache.commons.lang3.mutable.MutableLong;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.hadoop.fs.Path;
 import org.apache.parquet.schema.OriginalType;
 import org.apache.parquet.schema.PrimitiveType;
 
@@ -41,7 +42,7 @@ import static org.apache.drill.exec.store.parquet.metadata.Metadata_V3.ParquetTa
 public class ParquetGroupScanStatistics {
 
   // map from file names to maps of column name to partition value mappings
-  private Map<String, Map<SchemaPath, Object>> partitionValueMap;
+  private Map<Path, Map<SchemaPath, Object>> partitionValueMap;
   // only for partition columns : value is unique for each partition
   private Map<SchemaPath, TypeProtos.MajorType> partitionColTypeMap;
   // total number of non-null value for each column in parquet files
@@ -78,7 +79,7 @@ public class ParquetGroupScanStatistics {
     return rowCount;
   }
 
-  public Object getPartitionValue(String path, SchemaPath column) {
+  public Object getPartitionValue(Path path, SchemaPath column) {
     return partitionValueMap.get(path).get(column);
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRowGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRowGroupScan.java
index eabe2df..2513772 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRowGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRowGroupScan.java
@@ -37,6 +37,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
 
 // Class containing information for reading a single parquet row group from HDFS
 @JsonTypeName("parquet-row-group-scan")
@@ -45,7 +46,7 @@ public class ParquetRowGroupScan extends AbstractParquetRowGroupScan {
 
   private final ParquetFormatPlugin formatPlugin;
   private final ParquetFormatConfig formatConfig;
-  private final String selectionRoot;
+  private final Path selectionRoot;
 
   @JsonCreator
   public ParquetRowGroupScan(@JacksonInject StoragePluginRegistry registry,
@@ -55,7 +56,7 @@ public class ParquetRowGroupScan extends AbstractParquetRowGroupScan {
                              @JsonProperty("rowGroupReadEntries") LinkedList<RowGroupReadEntry> rowGroupReadEntries,
                              @JsonProperty("columns") List<SchemaPath> columns,
                              @JsonProperty("readerConfig") ParquetReaderConfig readerConfig,
-                             @JsonProperty("selectionRoot") String selectionRoot,
+                             @JsonProperty("selectionRoot") Path selectionRoot,
                              @JsonProperty("filter") LogicalExpression filter) throws ExecutionSetupException {
     this(userName,
         (ParquetFormatPlugin) registry.getFormatPlugin(Preconditions.checkNotNull(storageConfig), Preconditions.checkNotNull(formatConfig)),
@@ -71,7 +72,7 @@ public class ParquetRowGroupScan extends AbstractParquetRowGroupScan {
                              List<RowGroupReadEntry> rowGroupReadEntries,
                              List<SchemaPath> columns,
                              ParquetReaderConfig readerConfig,
-                             String selectionRoot,
+                             Path selectionRoot,
                              LogicalExpression filter) {
     super(userName, rowGroupReadEntries, columns, readerConfig, filter);
     this.formatPlugin = Preconditions.checkNotNull(formatPlugin, "Could not find format config for the given configuration");
@@ -90,7 +91,7 @@ public class ParquetRowGroupScan extends AbstractParquetRowGroupScan {
   }
 
   @JsonProperty
-  public String getSelectionRoot() {
+  public Path getSelectionRoot() {
     return selectionRoot;
   }
 
@@ -127,7 +128,7 @@ public class ParquetRowGroupScan extends AbstractParquetRowGroupScan {
 
   @Override
   public List<String> getPartitionValues(RowGroupReadEntry rowGroupReadEntry) {
-    return ColumnExplorer.listPartitionValues(rowGroupReadEntry.getPath(), selectionRoot);
+    return ColumnExplorer.listPartitionValues(rowGroupReadEntry.getPath(), selectionRoot, false);
   }
 }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
index f0ef639..8c91200 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
@@ -28,6 +28,7 @@ import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.server.options.OptionManager;
 import org.apache.drill.exec.store.dfs.DrillFileSystem;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
 
 import java.io.IOException;
 import java.util.List;
@@ -61,7 +62,7 @@ public class ParquetScanBatchCreator extends AbstractParquetScanBatchCreator imp
     }
 
     @Override
-    protected DrillFileSystem get(Configuration config, String path) throws ExecutionSetupException {
+    protected DrillFileSystem get(Configuration config, Path path) throws ExecutionSetupException {
       if (fs == null) {
         try {
           fs =  useAsyncPageReader ? operatorContext.newNonTrackingFileSystem(config) : operatorContext.newFileSystem(config);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/RowGroupInfo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/RowGroupInfo.java
index 1c9ce10..5fbadd6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/RowGroupInfo.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/RowGroupInfo.java
@@ -23,6 +23,7 @@ import org.apache.drill.exec.store.dfs.ReadEntryFromHDFS;
 import org.apache.drill.exec.store.dfs.easy.FileWork;
 import org.apache.drill.exec.store.schedule.CompleteWork;
 import org.apache.drill.exec.store.schedule.EndpointByteMap;
+import org.apache.hadoop.fs.Path;
 
 import java.util.List;
 
@@ -37,7 +38,7 @@ public class RowGroupInfo extends ReadEntryFromHDFS implements CompleteWork, Fil
   private long numRecordsToRead;
 
   @JsonCreator
-  public RowGroupInfo(@JsonProperty("path") String path,
+  public RowGroupInfo(@JsonProperty("path") Path path,
                       @JsonProperty("start") long start,
                       @JsonProperty("length") long length,
                       @JsonProperty("rowGroupIndex") int rowGroupIndex,
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/RowGroupReadEntry.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/RowGroupReadEntry.java
index 665179f..be3f50c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/RowGroupReadEntry.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/RowGroupReadEntry.java
@@ -22,6 +22,7 @@ import org.apache.drill.exec.store.dfs.ReadEntryFromHDFS;
 
 import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.hadoop.fs.Path;
 
 public class RowGroupReadEntry extends ReadEntryFromHDFS {
 
@@ -29,7 +30,7 @@ public class RowGroupReadEntry extends ReadEntryFromHDFS {
   private long numRecordsToRead;
 
   @JsonCreator
-  public RowGroupReadEntry(@JsonProperty("path") String path, @JsonProperty("start") long start,
+  public RowGroupReadEntry(@JsonProperty("path") Path path, @JsonProperty("start") long start,
                            @JsonProperty("length") long length, @JsonProperty("rowGroupIndex") int rowGroupIndex,
                            @JsonProperty("numRecordsToRead") long numRecordsToRead) {
     super(path, start, length);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetRecordReader.java
index 17cf8c4..ba4c493 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetRecordReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetRecordReader.java
@@ -90,7 +90,7 @@ public class ParquetRecordReader extends AbstractRecordReader {
   public boolean useBulkReader;
 
   @SuppressWarnings("unused")
-  private String name;
+  private Path name;
 
   public ParquetReaderStats parquetReaderStats = new ParquetReaderStats();
   private BatchReader batchReader;
@@ -123,44 +123,42 @@ public class ParquetRecordReader extends AbstractRecordReader {
   }
 
   public ParquetRecordReader(FragmentContext fragmentContext,
-      String path,
+      Path path,
       int rowGroupIndex,
       long numRecordsToRead,
       FileSystem fs,
       CodecFactory codecFactory,
       ParquetMetadata footer,
       List<SchemaPath> columns,
-      ParquetReaderUtility.DateCorruptionStatus dateCorruptionStatus) throws ExecutionSetupException {
-    this(fragmentContext, numRecordsToRead,
-         path, rowGroupIndex, fs, codecFactory, footer, columns, dateCorruptionStatus);
+      ParquetReaderUtility.DateCorruptionStatus dateCorruptionStatus) {
+    this(fragmentContext, numRecordsToRead, path, rowGroupIndex, fs, codecFactory, footer, columns, dateCorruptionStatus);
   }
 
   public ParquetRecordReader(FragmentContext fragmentContext,
-      String path,
+      Path path,
       int rowGroupIndex,
       FileSystem fs,
       CodecFactory codecFactory,
       ParquetMetadata footer,
       List<SchemaPath> columns,
-      ParquetReaderUtility.DateCorruptionStatus dateCorruptionStatus)
-      throws ExecutionSetupException {
-      this(fragmentContext, footer.getBlocks().get(rowGroupIndex).getRowCount(),
-           path, rowGroupIndex, fs, codecFactory, footer, columns, dateCorruptionStatus);
+      ParquetReaderUtility.DateCorruptionStatus dateCorruptionStatus) {
+    this(fragmentContext, footer.getBlocks().get(rowGroupIndex).getRowCount(), path, rowGroupIndex, fs, codecFactory,
+        footer, columns, dateCorruptionStatus);
   }
 
   public ParquetRecordReader(
       FragmentContext fragmentContext,
       long numRecordsToRead,
-      String path,
+      Path path,
       int rowGroupIndex,
       FileSystem fs,
       CodecFactory codecFactory,
       ParquetMetadata footer,
       List<SchemaPath> columns,
-      ParquetReaderUtility.DateCorruptionStatus dateCorruptionStatus) throws ExecutionSetupException {
+      ParquetReaderUtility.DateCorruptionStatus dateCorruptionStatus) {
 
     this.name = path;
-    this.hadoopPath = new Path(path);
+    this.hadoopPath = path;
     this.fileSystem = fs;
     this.codecFactory = codecFactory;
     this.rowGroupIndex = rowGroupIndex;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/Metadata.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/Metadata.java
index 0db007a..d0e2734 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/Metadata.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/Metadata.java
@@ -24,6 +24,7 @@ import com.fasterxml.jackson.databind.DeserializationFeature;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.module.SimpleModule;
 import com.fasterxml.jackson.module.afterburner.AfterburnerModule;
+import org.apache.drill.exec.serialization.PathSerDe;
 import org.apache.drill.exec.store.parquet.ParquetReaderConfig;
 import org.apache.drill.shaded.guava.com.google.common.base.Stopwatch;
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
@@ -82,6 +83,9 @@ import static org.apache.drill.exec.store.parquet.metadata.Metadata_V3.ParquetFi
 import static org.apache.drill.exec.store.parquet.metadata.Metadata_V3.ParquetTableMetadata_v3;
 import static org.apache.drill.exec.store.parquet.metadata.Metadata_V3.RowGroupMetadata_v3;
 
+/**
+ * This is an utility class, holder for Parquet Table Metadata and {@link ParquetReaderConfig}
+ */
 public class Metadata {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Metadata.class);
 
@@ -106,7 +110,7 @@ public class Metadata {
    * @param path path
    * @param readerConfig parquet reader configuration
    */
-  public static void createMeta(FileSystem fs, String path, ParquetReaderConfig readerConfig) throws IOException {
+  public static void createMeta(FileSystem fs, Path path, ParquetReaderConfig readerConfig) throws IOException {
     Metadata metadata = new Metadata(readerConfig);
     metadata.createMetaFilesRecursively(path, fs);
   }
@@ -208,13 +212,13 @@ public class Metadata {
    *         {@code path} directory).
    * @throws IOException if parquet metadata can't be serialized and written to the json file
    */
-  private Pair<ParquetTableMetadata_v3, ParquetTableMetadataDirs> createMetaFilesRecursively(final String path, FileSystem fs) throws IOException {
+  private Pair<ParquetTableMetadata_v3, ParquetTableMetadataDirs> createMetaFilesRecursively(final Path path, FileSystem fs) throws IOException {
     Stopwatch timer = logger.isDebugEnabled() ? Stopwatch.createStarted() : null;
     List<ParquetFileMetadata_v3> metaDataList = Lists.newArrayList();
-    List<String> directoryList = Lists.newArrayList();
+    List<Path> directoryList = Lists.newArrayList();
     ConcurrentHashMap<ColumnTypeMetadata_v3.Key, ColumnTypeMetadata_v3> columnTypeInfoSet =
         new ConcurrentHashMap<>();
-    Path p = new Path(path);
+    Path p = path;
     FileStatus fileStatus = fs.getFileStatus(p);
     assert fileStatus.isDirectory() : "Expected directory";
 
@@ -222,10 +226,10 @@ public class Metadata {
 
     for (final FileStatus file : DrillFileSystemUtil.listAll(fs, p, false)) {
       if (file.isDirectory()) {
-        ParquetTableMetadata_v3 subTableMetadata = (createMetaFilesRecursively(file.getPath().toString(), fs)).getLeft();
+        ParquetTableMetadata_v3 subTableMetadata = (createMetaFilesRecursively(file.getPath(), fs)).getLeft();
         metaDataList.addAll(subTableMetadata.files);
         directoryList.addAll(subTableMetadata.directories);
-        directoryList.add(file.getPath().toString());
+        directoryList.add(file.getPath());
         // Merge the schema from the child level into the current level
         //TODO: We need a merge method that merges two columns with the same name but different types
         columnTypeInfoSet.putAll(subTableMetadata.columnTypeInfo);
@@ -268,7 +272,7 @@ public class Metadata {
       ParquetTableMetadataDirs parquetTableMetadataDirs = new ParquetTableMetadataDirs(directoryList);
       return Pair.of(parquetTableMetadata, parquetTableMetadataDirs);
     }
-    List<String> emptyDirList = Lists.newArrayList();
+    List<Path> emptyDirList = new ArrayList<>();
     if (timer != null) {
       logger.debug("Creating metadata files recursively took {} ms", timer.elapsed(TimeUnit.MILLISECONDS));
       timer.stop();
@@ -495,7 +499,7 @@ public class Metadata {
 
       rowGroupMetadataList.add(rowGroupMeta);
     }
-    String path = Path.getPathWithoutSchemeAndAuthority(file.getPath()).toString();
+    Path path = Path.getPathWithoutSchemeAndAuthority(file.getPath());
 
     return new ParquetFileMetadata_v3(path, file.getLen(), rowGroupMetadataList);
   }
@@ -535,6 +539,8 @@ public class Metadata {
    *
    * @param parquetTableMetadata parquet table metadata
    * @param p file path
+   * @param fs Drill file system
+   * @throws IOException if metadata can't be serialized
    */
   private void writeFile(ParquetTableMetadata_v3 parquetTableMetadata, Path p, FileSystem fs) throws IOException {
     JsonFactory jsonFactory = new JsonFactory();
@@ -542,6 +548,7 @@ public class Metadata {
     jsonFactory.configure(JsonParser.Feature.AUTO_CLOSE_SOURCE, false);
     ObjectMapper mapper = new ObjectMapper(jsonFactory);
     SimpleModule module = new SimpleModule();
+    module.addSerializer(Path.class, new PathSerDe.Se());
     module.addSerializer(ColumnMetadata_v3.class, new ColumnMetadata_v3.Serializer());
     mapper.registerModule(module);
     OutputStream os = fs.create(p);
@@ -556,6 +563,7 @@ public class Metadata {
     jsonFactory.configure(JsonParser.Feature.AUTO_CLOSE_SOURCE, false);
     ObjectMapper mapper = new ObjectMapper(jsonFactory);
     SimpleModule module = new SimpleModule();
+    module.addSerializer(Path.class, new PathSerDe.Se());
     mapper.registerModule(module);
     OutputStream os = fs.create(p);
     mapper.writerWithDefaultPrettyPrinter().writeValue(os, parquetTableMetadataDirs);
@@ -602,7 +610,7 @@ public class Metadata {
         parquetTableMetadataDirs.updateRelativePaths(metadataParentDirPath);
         if (!alreadyCheckedModification && tableModified(parquetTableMetadataDirs.getDirectories(), path, metadataParentDir, metaContext, fs)) {
           parquetTableMetadataDirs =
-              (createMetaFilesRecursively(Path.getPathWithoutSchemeAndAuthority(path.getParent()).toString(), fs)).getRight();
+              (createMetaFilesRecursively(Path.getPathWithoutSchemeAndAuthority(path.getParent()), fs)).getRight();
           newMetadata = true;
         }
       } else {
@@ -616,7 +624,7 @@ public class Metadata {
         }
         if (!alreadyCheckedModification && tableModified(parquetTableMetadata.getDirectories(), path, metadataParentDir, metaContext, fs)) {
           parquetTableMetadata =
-              (createMetaFilesRecursively(Path.getPathWithoutSchemeAndAuthority(path.getParent()).toString(), fs)).getLeft();
+              (createMetaFilesRecursively(Path.getPathWithoutSchemeAndAuthority(path.getParent()), fs)).getLeft();
           newMetadata = true;
         }
 
@@ -647,9 +655,10 @@ public class Metadata {
    * @return true if metadata needs to be updated, false otherwise
    * @throws IOException if some resources are not accessible
    */
-  private boolean tableModified(List<String> directories, Path metaFilePath, Path parentDir, MetadataContext metaContext, FileSystem fs) throws IOException {
+  private boolean tableModified(List<Path> directories, Path metaFilePath, Path parentDir,
+                                MetadataContext metaContext, FileSystem fs) throws IOException {
     Stopwatch timer = logger.isDebugEnabled() ? Stopwatch.createStarted() : null;
-    metaContext.setStatus(parentDir.toUri().getPath());
+    metaContext.setStatus(parentDir);
     long metaFileModifyTime = fs.getFileStatus(metaFilePath).getModificationTime();
     FileStatus directoryStatus = fs.getFileStatus(parentDir);
     int numDirs = 1;
@@ -661,10 +670,10 @@ public class Metadata {
       }
       return true;
     }
-    for (String directory : directories) {
+    for (Path directory : directories) {
       numDirs++;
       metaContext.setStatus(directory);
-      directoryStatus = fs.getFileStatus(new Path(directory));
+      directoryStatus = fs.getFileStatus(directory);
       if (directoryStatus.getModificationTime() > metaFileModifyTime) {
         if (timer != null) {
           logger.debug("Directory {} was modified. Took {} ms to check modification time of {} directories",
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/MetadataBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/MetadataBase.java
index bed8be6..ee07470 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/MetadataBase.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/MetadataBase.java
@@ -20,6 +20,7 @@ package org.apache.drill.exec.store.parquet.metadata;
 import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonSubTypes;
 import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import org.apache.hadoop.fs.Path;
 import org.apache.parquet.schema.OriginalType;
 import org.apache.parquet.schema.PrimitiveType;
 
@@ -57,7 +58,7 @@ public class MetadataBase {
   public static abstract class ParquetTableMetadataBase {
 
     @JsonIgnore
-    public abstract List<String> getDirectories();
+    public abstract List<Path> getDirectories();
 
     @JsonIgnore public abstract List<? extends ParquetFileMetadata> getFiles();
 
@@ -83,7 +84,7 @@ public class MetadataBase {
   }
 
   public static abstract class ParquetFileMetadata {
-    @JsonIgnore public abstract String getPath();
+    @JsonIgnore public abstract Path getPath();
 
     @JsonIgnore public abstract Long getLength();
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/MetadataPathUtils.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/MetadataPathUtils.java
index 3e7c2ff..2794e2b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/MetadataPathUtils.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/MetadataPathUtils.java
@@ -21,6 +21,7 @@ import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
 import org.apache.drill.common.util.DrillVersionInfo;
 import org.apache.hadoop.fs.Path;
 
+import java.util.ArrayList;
 import java.util.List;
 
 import static org.apache.drill.exec.store.parquet.metadata.MetadataVersion.Constants.SUPPORTED_VERSIONS;
@@ -39,12 +40,12 @@ public class MetadataPathUtils {
    * @param baseDir base parent directory
    * @return list of absolute paths
    */
-  public static List<String> convertToAbsolutePaths(List<String> paths, String baseDir) {
+  public static List<Path> convertToAbsolutePaths(List<Path> paths, String baseDir) {
     if (!paths.isEmpty()) {
-      List<String> absolutePaths = Lists.newArrayList();
-      for (String relativePath : paths) {
-        String absolutePath = (new Path(relativePath).isAbsolute()) ? relativePath
-            : new Path(baseDir, relativePath).toUri().getPath();
+      List<Path> absolutePaths = Lists.newArrayList();
+      for (Path relativePath : paths) {
+        Path absolutePath = (relativePath.isAbsolute()) ? relativePath
+            : new Path(baseDir, relativePath);
         absolutePaths.add(absolutePath);
       }
       return absolutePaths;
@@ -64,10 +65,10 @@ public class MetadataPathUtils {
     if (!files.isEmpty()) {
       List<ParquetFileMetadata_v3> filesWithAbsolutePaths = Lists.newArrayList();
       for (ParquetFileMetadata_v3 file : files) {
-        Path relativePath = new Path(file.getPath());
+        Path relativePath = file.getPath();
         // create a new file if old one contains a relative path, otherwise use an old file
         ParquetFileMetadata_v3 fileWithAbsolutePath = (relativePath.isAbsolute()) ? file
-            : new ParquetFileMetadata_v3(new Path(baseDir, relativePath).toUri().getPath(), file.length, file.rowGroups);
+            : new ParquetFileMetadata_v3(new Path(baseDir, relativePath), file.length, file.rowGroups);
         filesWithAbsolutePaths.add(fileWithAbsolutePath);
       }
       return filesWithAbsolutePaths;
@@ -84,9 +85,9 @@ public class MetadataPathUtils {
    * @return parquet table metadata with relative paths for the files and directories
    */
   public static ParquetTableMetadata_v3 createMetadataWithRelativePaths(
-      ParquetTableMetadata_v3 tableMetadataWithAbsolutePaths, String baseDir) {
-    List<String> directoriesWithRelativePaths = Lists.newArrayList();
-    for (String directory : tableMetadataWithAbsolutePaths.getDirectories()) {
+      ParquetTableMetadata_v3 tableMetadataWithAbsolutePaths, Path baseDir) {
+    List<Path> directoriesWithRelativePaths = new ArrayList<>();
+    for (Path directory : tableMetadataWithAbsolutePaths.getDirectories()) {
       directoriesWithRelativePaths.add(relativize(baseDir, directory));
     }
     List<ParquetFileMetadata_v3> filesWithRelativePaths = Lists.newArrayList();
@@ -105,9 +106,9 @@ public class MetadataPathUtils {
    * @param baseDir base path (the part of the Path, which should be cut off from child path)
    * @return relative path
    */
-  public static String relativize(String baseDir, String childPath) {
-    Path fullPathWithoutSchemeAndAuthority = Path.getPathWithoutSchemeAndAuthority(new Path(childPath));
-    Path basePathWithoutSchemeAndAuthority = Path.getPathWithoutSchemeAndAuthority(new Path(baseDir));
+  public static Path relativize(Path baseDir, Path childPath) {
+    Path fullPathWithoutSchemeAndAuthority = Path.getPathWithoutSchemeAndAuthority(childPath);
+    Path basePathWithoutSchemeAndAuthority = Path.getPathWithoutSchemeAndAuthority(baseDir);
 
     // Since hadoop Path hasn't relativize() we use uri.relativize() to get relative path
     Path relativeFilePath = new Path(basePathWithoutSchemeAndAuthority.toUri()
@@ -116,7 +117,7 @@ public class MetadataPathUtils {
       throw new IllegalStateException(String.format("Path %s is not a subpath of %s.",
           basePathWithoutSchemeAndAuthority.toUri().getPath(), fullPathWithoutSchemeAndAuthority.toUri().getPath()));
     }
-    return relativeFilePath.toUri().getPath();
+    return relativeFilePath;
   }
 
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/Metadata_V1.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/Metadata_V1.java
index 92feb5f..4b0dca8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/Metadata_V1.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/Metadata_V1.java
@@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
 import org.apache.drill.common.expression.SchemaPath;
+import org.apache.hadoop.fs.Path;
 import org.apache.parquet.io.api.Binary;
 import org.apache.parquet.schema.OriginalType;
 import org.apache.parquet.schema.PrimitiveType;
@@ -43,19 +44,19 @@ public class Metadata_V1 {
     @JsonProperty(value = "metadata_version", access = JsonProperty.Access.WRITE_ONLY) private String metadataVersion;
     @JsonProperty
     List<ParquetFileMetadata_v1> files;
-    @JsonProperty List<String> directories;
+    @JsonProperty List<Path> directories;
 
     public ParquetTableMetadata_v1() {
     }
 
-    public ParquetTableMetadata_v1(String metadataVersion, List<ParquetFileMetadata_v1> files, List<String> directories) {
+    public ParquetTableMetadata_v1(String metadataVersion, List<ParquetFileMetadata_v1> files, List<Path> directories) {
       this.metadataVersion = metadataVersion;
       this.files = files;
       this.directories = directories;
     }
 
     @JsonIgnore
-    @Override public List<String> getDirectories() {
+    @Override public List<Path> getDirectories() {
       return directories;
     }
 
@@ -114,7 +115,7 @@ public class Metadata_V1 {
    */
   public static class ParquetFileMetadata_v1 extends ParquetFileMetadata {
     @JsonProperty
-    public String path;
+    public Path path;
     @JsonProperty
     public Long length;
     @JsonProperty
@@ -123,7 +124,7 @@ public class Metadata_V1 {
     public ParquetFileMetadata_v1() {
     }
 
-    public ParquetFileMetadata_v1(String path, Long length, List<RowGroupMetadata_v1> rowGroups) {
+    public ParquetFileMetadata_v1(Path path, Long length, List<RowGroupMetadata_v1> rowGroups) {
       this.path = path;
       this.length = length;
       this.rowGroups = rowGroups;
@@ -134,7 +135,7 @@ public class Metadata_V1 {
       return String.format("path: %s rowGroups: %s", path, rowGroups);
     }
 
-    @JsonIgnore @Override public String getPath() {
+    @JsonIgnore @Override public Path getPath() {
       return path;
     }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/Metadata_V2.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/Metadata_V2.java
index 7eddc12..a78fca4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/Metadata_V2.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/Metadata_V2.java
@@ -28,6 +28,7 @@ import com.fasterxml.jackson.databind.JsonDeserializer;
 import com.fasterxml.jackson.databind.JsonSerializer;
 import com.fasterxml.jackson.databind.KeyDeserializer;
 import com.fasterxml.jackson.databind.SerializerProvider;
+import org.apache.hadoop.fs.Path;
 import org.apache.parquet.io.api.Binary;
 import org.apache.parquet.schema.OriginalType;
 import org.apache.parquet.schema.PrimitiveType;
@@ -59,7 +60,7 @@ public class Metadata_V2 {
     @JsonProperty public ConcurrentHashMap<ColumnTypeMetadata_v2.Key, ColumnTypeMetadata_v2> columnTypeInfo;
     @JsonProperty
     List<ParquetFileMetadata_v2> files;
-    @JsonProperty List<String> directories;
+    @JsonProperty List<Path> directories;
     @JsonProperty String drillVersion;
 
     public ParquetTableMetadata_v2() {
@@ -71,7 +72,7 @@ public class Metadata_V2 {
     }
 
     public ParquetTableMetadata_v2(String metadataVersion, ParquetTableMetadataBase parquetTable,
-                                   List<ParquetFileMetadata_v2> files, List<String> directories, String drillVersion) {
+                                   List<ParquetFileMetadata_v2> files, List<Path> directories, String drillVersion) {
       this.metadataVersion = metadataVersion;
       this.files = files;
       this.directories = directories;
@@ -79,7 +80,7 @@ public class Metadata_V2 {
       this.drillVersion = drillVersion;
     }
 
-    public ParquetTableMetadata_v2(String metadataVersion, List<ParquetFileMetadata_v2> files, List<String> directories,
+    public ParquetTableMetadata_v2(String metadataVersion, List<ParquetFileMetadata_v2> files, List<Path> directories,
                                    ConcurrentHashMap<ColumnTypeMetadata_v2.Key, ColumnTypeMetadata_v2> columnTypeInfo, String drillVersion) {
       this.metadataVersion = metadataVersion;
       this.files = files;
@@ -93,7 +94,7 @@ public class Metadata_V2 {
     }
 
     @JsonIgnore
-    @Override public List<String> getDirectories() {
+    @Override public List<Path> getDirectories() {
       return directories;
     }
 
@@ -152,14 +153,14 @@ public class Metadata_V2 {
    * Struct which contains the metadata for a single parquet file
    */
   public static class ParquetFileMetadata_v2 extends ParquetFileMetadata {
-    @JsonProperty public String path;
+    @JsonProperty public Path path;
     @JsonProperty public Long length;
     @JsonProperty public List<RowGroupMetadata_v2> rowGroups;
 
     public ParquetFileMetadata_v2() {
     }
 
-    public ParquetFileMetadata_v2(String path, Long length, List<RowGroupMetadata_v2> rowGroups) {
+    public ParquetFileMetadata_v2(Path path, Long length, List<RowGroupMetadata_v2> rowGroups) {
       this.path = path;
       this.length = length;
       this.rowGroups = rowGroups;
@@ -169,7 +170,7 @@ public class Metadata_V2 {
       return String.format("path: %s rowGroups: %s", path, rowGroups);
     }
 
-    @JsonIgnore @Override public String getPath() {
+    @JsonIgnore @Override public Path getPath() {
       return path;
     }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/Metadata_V3.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/Metadata_V3.java
index 4bb07f7..a5ff897 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/Metadata_V3.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/Metadata_V3.java
@@ -25,6 +25,7 @@ import com.fasterxml.jackson.databind.JsonSerializer;
 import com.fasterxml.jackson.databind.KeyDeserializer;
 import com.fasterxml.jackson.databind.SerializerProvider;
 import org.apache.drill.common.expression.SchemaPath;
+import org.apache.hadoop.fs.Path;
 import org.apache.parquet.io.api.Binary;
 import org.apache.parquet.schema.OriginalType;
 import org.apache.parquet.schema.PrimitiveType;
@@ -54,7 +55,7 @@ public class Metadata_V3 {
     @JsonProperty public ConcurrentHashMap<ColumnTypeMetadata_v3.Key, ColumnTypeMetadata_v3> columnTypeInfo;
     @JsonProperty
     List<ParquetFileMetadata_v3> files;
-    @JsonProperty List<String> directories;
+    @JsonProperty List<Path> directories;
     @JsonProperty String drillVersion;
 
     /**
@@ -74,7 +75,7 @@ public class Metadata_V3 {
     }
 
     public ParquetTableMetadata_v3(String metadataVersion, ParquetTableMetadataBase parquetTable,
-                                   List<ParquetFileMetadata_v3> files, List<String> directories, String drillVersion) {
+                                   List<ParquetFileMetadata_v3> files, List<Path> directories, String drillVersion) {
       this.metadataVersion = metadataVersion;
       this.files = files;
       this.directories = directories;
@@ -82,7 +83,7 @@ public class Metadata_V3 {
       this.drillVersion = drillVersion;
     }
 
-    public ParquetTableMetadata_v3(String metadataVersion, List<ParquetFileMetadata_v3> files, List<String> directories,
+    public ParquetTableMetadata_v3(String metadataVersion, List<ParquetFileMetadata_v3> files, List<Path> directories,
                                    ConcurrentHashMap<ColumnTypeMetadata_v3.Key, ColumnTypeMetadata_v3> columnTypeInfo,
                                    String drillVersion) {
       this.metadataVersion = metadataVersion;
@@ -97,7 +98,7 @@ public class Metadata_V3 {
     }
 
     @JsonIgnore
-    @Override public List<String> getDirectories() {
+    @Override public List<Path> getDirectories() {
       return directories;
     }
 
@@ -168,14 +169,14 @@ public class Metadata_V3 {
    * Struct which contains the metadata for a single parquet file
    */
   public static class ParquetFileMetadata_v3 extends ParquetFileMetadata {
-    @JsonProperty public String path;
+    @JsonProperty public Path path;
     @JsonProperty public Long length;
     @JsonProperty public List<RowGroupMetadata_v3> rowGroups;
 
     public ParquetFileMetadata_v3() {
     }
 
-    public ParquetFileMetadata_v3(String path, Long length, List<RowGroupMetadata_v3> rowGroups) {
+    public ParquetFileMetadata_v3(Path path, Long length, List<RowGroupMetadata_v3> rowGroups) {
       this.path = path;
       this.length = length;
       this.rowGroups = rowGroups;
@@ -185,7 +186,7 @@ public class Metadata_V3 {
       return String.format("path: %s rowGroups: %s", path, rowGroups);
     }
 
-    @JsonIgnore @Override public String getPath() {
+    @JsonIgnore @Override public Path getPath() {
       return path;
     }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/ParquetTableMetadataDirs.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/ParquetTableMetadataDirs.java
index 186f534..b1fd7f2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/ParquetTableMetadataDirs.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/ParquetTableMetadataDirs.java
@@ -19,24 +19,25 @@ package org.apache.drill.exec.store.parquet.metadata;
 
 import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.hadoop.fs.Path;
 
 import java.util.List;
 
 public class ParquetTableMetadataDirs {
 
   @JsonProperty
-  List<String> directories;
+  List<Path> directories;
 
   public ParquetTableMetadataDirs() {
     // default constructor needed for deserialization
   }
 
-  public ParquetTableMetadataDirs(List<String> directories) {
+  public ParquetTableMetadataDirs(List<Path> directories) {
     this.directories = directories;
   }
 
   @JsonIgnore
-  public List<String> getDirectories() {
+  public List<Path> getDirectories() {
     return directories;
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet2/DrillParquetReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet2/DrillParquetReader.java
index 09c016a..5c49e04 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet2/DrillParquetReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet2/DrillParquetReader.java
@@ -235,7 +235,7 @@ public class DrillParquetReader extends AbstractRecordReader {
         paths.put(md.getPath(), md);
       }
 
-      Path filePath = new Path(entry.getPath());
+      Path filePath = entry.getPath();
 
       BlockMetaData blockMetaData = footer.getBlocks().get(entry.getRowGroupIndex());
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapRecordReader.java
index d688f3b..aef56a3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapRecordReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapRecordReader.java
@@ -92,11 +92,11 @@ public class PcapRecordReader extends AbstractRecordReader {
         .build();
   }
 
-  public PcapRecordReader(final String pathToFile,
+  public PcapRecordReader(final Path pathToFile,
                           final FileSystem fileSystem,
                           final List<SchemaPath> projectedColumns) {
     this.fs = fileSystem;
-    this.pathToFile = fs.makeQualified(new Path(pathToFile));
+    this.pathToFile = fs.makeQualified(pathToFile);
     this.projectedColumns = projectedColumns;
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcapng/PcapngRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcapng/PcapngRecordReader.java
index b1c5f24..0ad234d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcapng/PcapngRecordReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcapng/PcapngRecordReader.java
@@ -65,11 +65,11 @@ public class PcapngRecordReader extends AbstractRecordReader {
 
   private Iterator<IPcapngType> it;
 
-  public PcapngRecordReader(final String pathToFile,
+  public PcapngRecordReader(final Path pathToFile,
                             final FileSystem fileSystem,
                             final List<SchemaPath> columns) {
     this.fs = fileSystem;
-    this.pathToFile = fs.makeQualified(new Path(pathToFile));
+    this.pathToFile = fs.makeQualified(pathToFile);
     this.columns = columns;
     setColumns(columns);
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/BlockMapBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/BlockMapBuilder.java
index 7de31a0..6bc7bb0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/BlockMapBuilder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/BlockMapBuilder.java
@@ -116,8 +116,8 @@ public class BlockMapBuilder {
         try {
           ImmutableRangeMap<Long, BlockLocation> rangeMap = getBlockMap(status);
           for (Entry<Range<Long>, BlockLocation> l : rangeMap.asMapOfRanges().entrySet()) {
-            work.add(new CompleteFileWork(getEndpointByteMap(noDrillbitHosts, new FileStatusWork(status)), l.getValue().getOffset(), l.getValue().getLength(), status.getPath()
-              .toString()));
+            work.add(new CompleteFileWork(getEndpointByteMap(noDrillbitHosts, new FileStatusWork(status)),
+                l.getValue().getOffset(), l.getValue().getLength(), status.getPath()));
           }
         } catch (IOException e) {
           logger.warn("failure while generating file work.", e);
@@ -127,7 +127,8 @@ public class BlockMapBuilder {
 
 
       if (!blockify || error || compressed(status)) {
-        work.add(new CompleteFileWork(getEndpointByteMap(noDrillbitHosts, new FileStatusWork(status)), 0, status.getLen(), status.getPath().toString()));
+        work.add(new CompleteFileWork(getEndpointByteMap(noDrillbitHosts, new FileStatusWork(status)), 0,
+            status.getLen(), status.getPath()));
       }
 
       // This if-condition is specific for empty CSV file
@@ -135,7 +136,8 @@ public class BlockMapBuilder {
       // And if this CSV file is empty, rangeMap would be empty also
       // Therefore, at the point before this if-condition, work would not be populated
       if(work.isEmpty()) {
-        work.add(new CompleteFileWork(getEndpointByteMap(noDrillbitHosts, new FileStatusWork(status)), 0, 0, status.getPath().toString()));
+        work.add(new CompleteFileWork(getEndpointByteMap(noDrillbitHosts, new FileStatusWork(status)), 0, 0,
+            status.getPath()));
       }
 
       if (noDrillbitHosts != null) {
@@ -162,8 +164,8 @@ public class BlockMapBuilder {
     }
 
     @Override
-    public String getPath() {
-      return status.getPath().toString();
+    public Path getPath() {
+      return status.getPath();
     }
 
     @Override
@@ -231,20 +233,20 @@ public class BlockMapBuilder {
    */
   public EndpointByteMap getEndpointByteMap(Set<String> noDrillbitHosts, FileWork work) throws IOException {
     Stopwatch watch = Stopwatch.createStarted();
-    Path fileName = new Path(work.getPath());
+    Path fileName = work.getPath();
 
 
-    ImmutableRangeMap<Long,BlockLocation> blockMap = getBlockMap(fileName);
+    ImmutableRangeMap<Long, BlockLocation> blockMap = getBlockMap(fileName);
     EndpointByteMapImpl endpointByteMap = new EndpointByteMapImpl();
     long start = work.getStart();
     long end = start + work.getLength();
     Range<Long> rowGroupRange = Range.closedOpen(start, end);
 
     // Find submap of ranges that intersect with the rowGroup
-    ImmutableRangeMap<Long,BlockLocation> subRangeMap = blockMap.subRangeMap(rowGroupRange);
+    ImmutableRangeMap<Long, BlockLocation> subRangeMap = blockMap.subRangeMap(rowGroupRange);
 
     // Iterate through each block in this submap and get the host for the block location
-    for (Map.Entry<Range<Long>,BlockLocation> block : subRangeMap.asMapOfRanges().entrySet()) {
+    for (Map.Entry<Range<Long>, BlockLocation> block : subRangeMap.asMapOfRanges().entrySet()) {
       String[] hosts;
       Range<Long> blockRange = block.getKey();
       try {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/CompleteFileWork.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/CompleteFileWork.java
index 04c4eb0..4b0402b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/CompleteFileWork.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/CompleteFileWork.java
@@ -21,16 +21,17 @@ import org.apache.drill.exec.store.dfs.easy.FileWork;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.hadoop.fs.Path;
 
 public class CompleteFileWork implements FileWork, CompleteWork {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(CompleteFileWork.class);
 
   private long start;
   private long length;
-  private String path;
+  private Path path;
   private EndpointByteMap byteMap;
 
-  public CompleteFileWork(EndpointByteMap byteMap, long start, long length, String path) {
+  public CompleteFileWork(EndpointByteMap byteMap, long start, long length, Path path) {
     super();
     this.start = start;
     this.length = length;
@@ -69,7 +70,7 @@ public class CompleteFileWork implements FileWork, CompleteWork {
   }
 
   @Override
-  public String getPath() {
+  public Path getPath() {
     return path;
   }
 
@@ -87,22 +88,28 @@ public class CompleteFileWork implements FileWork, CompleteWork {
     return new FileWorkImpl(start, length, path);
   }
 
-  public static class FileWorkImpl implements FileWork{
+  @Override
+  public String toString() {
+    return String.format("File: %s start: %d length: %d", path, start, length);
+  }
+
+  public static class FileWorkImpl implements FileWork {
+
+    private long start;
+    private long length;
+    private Path path;
 
     @JsonCreator
-    public FileWorkImpl(@JsonProperty("start") long start, @JsonProperty("length") long length, @JsonProperty("path") String path) {
-      super();
+    public FileWorkImpl(@JsonProperty("start") long start,
+                        @JsonProperty("length") long length,
+                        @JsonProperty("path") Path path) {
       this.start = start;
       this.length = length;
       this.path = path;
     }
 
-    public long start;
-    public long length;
-    public String path;
-
     @Override
-    public String getPath() {
+    public Path getPath() {
       return path;
     }
 
@@ -116,10 +123,13 @@ public class CompleteFileWork implements FileWork, CompleteWork {
       return length;
     }
 
-  }
-
-  @Override
-  public String toString() {
-    return String.format("File: %s start: %d length: %d", path, start, length);
+    @Override
+    public String toString() {
+      return "FileWorkImpl{" +
+          "start=" + start +
+          ", length=" + length +
+          ", path=" + path +
+          '}';
+    }
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/util/DrillFileSystemUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/util/DrillFileSystemUtil.java
index fcee3b0..bfb83e0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/util/DrillFileSystemUtil.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/util/DrillFileSystemUtil.java
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.util;
 
 import org.apache.drill.exec.store.dfs.DrillFileSystem;
+import org.apache.drill.shaded.guava.com.google.common.base.Strings;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -133,4 +134,14 @@ public class DrillFileSystemUtil {
     return FileSystemUtil.listAllSafe(fs, path, recursive, FileSystemUtil.mergeFilters(DRILL_SYSTEM_FILTER, filters));
   }
 
+  /**
+   * Safely creates Hadoop Path for null and empty String paths
+   *
+   * @param path String path, which can be null or empty
+   * @return Hadoop Path. Root - for empty or null path
+   */
+  public static Path createPathSafe(String path) {
+    return Strings.isNullOrEmpty(path) ? new Path("/") : new Path(path);
+  }
+
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/PlanTestBase.java b/exec/java-exec/src/test/java/org/apache/drill/PlanTestBase.java
index 9e0d95c..247d784 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/PlanTestBase.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/PlanTestBase.java
@@ -143,8 +143,7 @@ public class PlanTestBase extends BaseTestQuery {
    *                     planning process throws an exception
    */
   public static void testPlanWithAttributesMatchingPatterns(String query, String[] expectedPatterns,
-                                                            String[] excludedPatterns)
-          throws Exception {
+                                                            String[] excludedPatterns) throws Exception {
     final String plan = getPlanInString("EXPLAIN PLAN INCLUDING ALL ATTRIBUTES for " +
             QueryTestUtil.normalizeQuery(query), OPTIQ_FORMAT);
 
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/TestPathSerialization.java b/exec/java-exec/src/test/java/org/apache/drill/exec/TestPathSerialization.java
new file mode 100644
index 0000000..cb93565
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/TestPathSerialization.java
@@ -0,0 +1,65 @@
+/*
+ * 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;
+
+
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import org.apache.drill.exec.serialization.PathSerDe;
+import org.apache.drill.exec.store.schedule.CompleteFileWork;
+import org.apache.drill.test.DrillTest;
+import org.apache.hadoop.fs.Path;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.junit.Assert.assertThat;
+
+public class TestPathSerialization extends DrillTest {
+
+  @Test
+  public void testDeSerializingWithJsonCreator() throws IOException {
+
+    String jsonString = "{\"start\": 1, \"length\": 2, \"path\": \"/tmp/drill/test\"}";
+
+    SimpleModule module = new SimpleModule();
+    module.addSerializer(Path.class, new PathSerDe.Se());
+    objectMapper.registerModule(module);
+
+    CompleteFileWork.FileWorkImpl bean = objectMapper.readValue(jsonString, CompleteFileWork.FileWorkImpl.class);
+
+    assertThat(bean.getStart() == 1,  equalTo( true ));
+    assertThat(bean.getLength() == 2, equalTo( true ));
+    assertThat(bean.getPath().equals(new Path("/tmp/drill/test")), equalTo( true ));
+  }
+
+  @Test
+  public void testHadoopPathSerDe() throws IOException {
+    CompleteFileWork.FileWorkImpl fileWork = new CompleteFileWork.FileWorkImpl(5, 6, new Path("/tmp"));
+    SimpleModule module = new SimpleModule();
+    module.addSerializer(Path.class, new PathSerDe.Se());
+    objectMapper.registerModule(module);
+
+    CompleteFileWork.FileWorkImpl bean =
+        objectMapper.readValue(objectMapper.writeValueAsString(fileWork), CompleteFileWork.FileWorkImpl.class);
+
+    assertThat(bean.getStart() == 5,  equalTo( true ));
+    assertThat(bean.getLength() == 6, equalTo( true ));
+    assertThat(bean.getPath().equals(new Path("/tmp")), equalTo( true ));
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestFileScanFramework.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestFileScanFramework.java
index 2fdf3e6..b05bb28 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestFileScanFramework.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestFileScanFramework.java
@@ -85,7 +85,7 @@ public class TestFileScanFramework extends SubOperatorTest {
     }
 
     @Override
-    public String getPath() { return path.toString(); }
+    public Path getPath() { return path; }
 
     @Override
     public long getStart() { return 0; }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/MiniPlanUnitTestBase.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/MiniPlanUnitTestBase.java
index 9a2bb1f..911a097 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/MiniPlanUnitTestBase.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/MiniPlanUnitTestBase.java
@@ -32,6 +32,7 @@ import org.apache.drill.exec.store.dfs.DrillFileSystem;
 import org.apache.drill.exec.store.parquet.ParquetDirectByteBufferAllocator;
 import org.apache.drill.exec.store.parquet.ParquetReaderUtility;
 import org.apache.drill.exec.store.parquet.columnreaders.ParquetRecordReader;
+import org.apache.drill.test.LegacyOperatorTestBuilder;
 import org.apache.drill.test.PhysicalOpUnitTestBase;
 import org.apache.hadoop.fs.Path;
 import org.apache.parquet.hadoop.CodecFactory;
@@ -358,7 +359,7 @@ public class MiniPlanUnitTestBase extends PhysicalOpUnitTestBase {
    */
   public class JsonScanBuilder extends ScanPopBuider<JsonScanBuilder> {
     List<String> jsonBatches = null;
-    List<String> inputPaths = Collections.emptyList();
+    List<Path> inputPaths = Collections.emptyList();
 
     public JsonScanBuilder(PopBuilder parent) {
       super(parent);
@@ -373,7 +374,7 @@ public class MiniPlanUnitTestBase extends PhysicalOpUnitTestBase {
       return this;
     }
 
-    public JsonScanBuilder inputPaths(List<String> inputPaths) {
+    public JsonScanBuilder inputPaths(List<Path> inputPaths) {
       this.inputPaths = inputPaths;
       return this;
     }
@@ -412,7 +413,7 @@ public class MiniPlanUnitTestBase extends PhysicalOpUnitTestBase {
    * Builder for parquet Scan RecordBatch.
    */
   public class ParquetScanBuilder extends ScanPopBuider<ParquetScanBuilder> {
-    List<String> inputPaths = Collections.emptyList();
+    List<Path> inputPaths = Collections.emptyList();
 
     public ParquetScanBuilder() {
       super();
@@ -422,7 +423,7 @@ public class MiniPlanUnitTestBase extends PhysicalOpUnitTestBase {
       super(parent);
     }
 
-    public ParquetScanBuilder inputPaths(List<String> inputPaths) {
+    public ParquetScanBuilder inputPaths(List<Path> inputPaths) {
       this.inputPaths = inputPaths;
       return this;
     }
@@ -443,8 +444,8 @@ public class MiniPlanUnitTestBase extends PhysicalOpUnitTestBase {
     private RecordBatch getScanBatch() throws Exception {
       List<RecordReader> readers = new LinkedList<>();
 
-      for (String path : inputPaths) {
-        ParquetMetadata footer = ParquetFileReader.readFooter(fs.getConf(), new Path(path));
+      for (Path path : inputPaths) {
+        ParquetMetadata footer = ParquetFileReader.readFooter(fs.getConf(), path);
 
         for (int i = 0; i < footer.getBlocks().size(); i++) {
           readers.add(new ParquetRecordReader(fragContext,
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestMiniPlan.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestMiniPlan.java
index cc259cc..a1b7001 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestMiniPlan.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestMiniPlan.java
@@ -32,6 +32,7 @@ import org.apache.drill.exec.record.metadata.SchemaBuilder;
 import org.apache.drill.exec.store.dfs.DrillFileSystem;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -59,11 +60,11 @@ public class TestMiniPlan extends MiniPlanUnitTestBase {
   @Test
   public void testSimpleParquetScan() throws Exception {
     String file = DrillFileUtils.getResourceAsFile("/tpchmulti/region/01.parquet").toURI().toString();
-
+    List<Path> filePath = Collections.singletonList(new Path(file));
     RecordBatch scanBatch = new ParquetScanBuilder()
         .fileSystem(fs)
         .columnsToRead("R_REGIONKEY")
-        .inputPaths(Lists.newArrayList(file))
+        .inputPaths(filePath)
         .build();
 
     BatchSchema expectedSchema = new SchemaBuilder()
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestNullInputMiniPlan.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestNullInputMiniPlan.java
index bdff80b..69a4214 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestNullInputMiniPlan.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestNullInputMiniPlan.java
@@ -41,6 +41,7 @@ import org.apache.drill.exec.record.metadata.SchemaBuilder;
 import org.apache.drill.exec.store.dfs.DrillFileSystem;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
@@ -272,11 +273,12 @@ public class TestNullInputMiniPlan extends MiniPlanUnitTestBase{
     RecordBatch left = createScanBatchFromJson(SINGLE_EMPTY_JSON);
 
     String file = DrillFileUtils.getResourceAsFile("/tpchmulti/region/01.parquet").toURI().toString();
+    List<Path> filePath = Collections.singletonList(new Path(file));
 
     RecordBatch scanBatch = new ParquetScanBuilder()
         .fileSystem(fs)
         .columnsToRead("R_REGIONKEY")
-        .inputPaths(Lists.newArrayList(file))
+        .inputPaths(filePath)
         .build();
 
     RecordBatch projectBatch = new PopBuilder()
@@ -554,10 +556,10 @@ public class TestNullInputMiniPlan extends MiniPlanUnitTestBase{
   }
 
   private RecordBatch createScanBatchFromJson(String... resourcePaths) throws Exception {
-    List<String> inputPaths = new ArrayList<>();
+    List<Path> inputPaths = new ArrayList<>();
 
     for (String resource : resourcePaths) {
-      inputPaths.add(DrillFileUtils.getResourceAsFile(resource).toURI().toString());
+      inputPaths.add(new Path(DrillFileUtils.getResourceAsFile(resource).toURI()));
     }
 
     RecordBatch scanBatch = new JsonScanBuilder()
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/CachedSingleFileSystem.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/CachedSingleFileSystem.java
index 6bd7e4d..3db9256 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/CachedSingleFileSystem.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/CachedSingleFileSystem.java
@@ -41,16 +41,16 @@ import org.apache.hadoop.util.Progressable;
 public class CachedSingleFileSystem extends FileSystem {
 
   private ByteBuf file;
-  private String path;
+  private Path path;
 
-  public CachedSingleFileSystem(String path) throws IOException {
+  public CachedSingleFileSystem(Path path) throws IOException {
     this.path = path;
-    File f = new File(path);
+    File f = new File(path.toUri().getPath());
     long length = f.length();
     if (length > Integer.MAX_VALUE) {
       throw new UnsupportedOperationException("Cached file system only supports files of less than 2GB.");
     }
-    try (InputStream is = new BufferedInputStream(new FileInputStream(path))) {
+    try (InputStream is = new BufferedInputStream(new FileInputStream(path.toUri().getPath()))) {
       byte[] buffer = new byte[64*1024];
       this.file = UnpooledByteBufAllocator.DEFAULT.directBuffer((int) length);
       int read;
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/dfs/TestDFSPartitionLocation.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/dfs/TestDFSPartitionLocation.java
new file mode 100644
index 0000000..067abec
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/dfs/TestDFSPartitionLocation.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.store.dfs;
+
+import org.apache.drill.exec.planner.DFSFilePartitionLocation;
+import org.apache.drill.test.DrillTest;
+import org.apache.hadoop.fs.Path;
+import org.junit.Test;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+public class TestDFSPartitionLocation extends DrillTest {
+
+  private static final Path SELECTION_ROOT = new Path("/tmp/drill");
+  private static final Path PARTITION = new Path("/tmp/drill/test_table/first_dir/second_dir/");
+
+  @Test
+  public void testDFSFilePartitionLocation() {
+    Path file = new Path(PARTITION, "0_0_0.parquet");
+    DFSFilePartitionLocation dfsPartition = new DFSFilePartitionLocation(4, SELECTION_ROOT, file, false);
+    checkSubdirectories(dfsPartition, file);
+  }
+
+  @Test
+  public void testDFSDirectoryPartitionLocation() {
+    DFSFilePartitionLocation dfsPartition = new DFSFilePartitionLocation(4, SELECTION_ROOT, PARTITION, true);
+    checkSubdirectories(dfsPartition, PARTITION);
+  }
+
+  private void checkSubdirectories(DFSFilePartitionLocation dfsPartition, Path partition) {
+    assertArrayEquals("Wrong partition dirs",  new String[]{"test_table", "first_dir", "second_dir", null}, dfsPartition.getDirs());
+    assertEquals("Wrong partition value","test_table", dfsPartition.getPartitionValue(0));
+    assertEquals("Wrong partition value", "first_dir", dfsPartition.getPartitionValue(1));
+    assertEquals("Wrong partition value", "second_dir", dfsPartition.getPartitionValue(2));
+    assertEquals("Wrong partition location", partition, dfsPartition.getEntirePartitionLocation());
+  }
+
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/dfs/TestFileSelection.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/dfs/TestFileSelection.java
index b1f233e..b853b9b 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/dfs/TestFileSelection.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/dfs/TestFileSelection.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertNull;
 
 import java.util.List;
 
+import org.apache.drill.exec.util.DrillFileSystemUtil;
 import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
 import org.apache.drill.test.BaseTestQuery;
 import org.apache.hadoop.fs.FileStatus;
@@ -30,7 +31,7 @@ import org.junit.Test;
 
 public class TestFileSelection extends BaseTestQuery {
   private static final List<FileStatus> EMPTY_STATUSES = ImmutableList.of();
-  private static final List<String> EMPTY_FILES = ImmutableList.of();
+  private static final List<Path> EMPTY_FILES = ImmutableList.of();
   private static final String EMPTY_ROOT = "";
 
   @Test
@@ -38,8 +39,8 @@ public class TestFileSelection extends BaseTestQuery {
     for (final Object statuses : new Object[] { null, EMPTY_STATUSES}) {
       for (final Object files : new Object[]{null, EMPTY_FILES}) {
         for (final Object root : new Object[]{null, EMPTY_ROOT}) {
-          final FileSelection selection = FileSelection.create((List<FileStatus>) statuses, (List<String>) files,
-              (String)root);
+          FileSelection selection = FileSelection.create((List<FileStatus>) statuses, (List<Path>) files,
+                  DrillFileSystemUtil.createPathSafe((String) root));
           assertNull(selection);
         }
       }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
index 1bd90b3..16b25ce 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
@@ -610,13 +610,13 @@ public class ParquetRecordReaderTest extends BaseTestQuery {
     final FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
     final FragmentContextImpl context = new FragmentContextImpl(bitContext, BitControl.PlanFragment.getDefaultInstance(), connection, registry);
 
-    final String fileName = "/tmp/parquet_test_performance.parquet";
+    final Path fileName = new Path("/tmp/parquet_test_performance.parquet");
     final HashMap<String, FieldInfo> fields = new HashMap<>();
     final ParquetTestProperties props = new ParquetTestProperties(1, 20 * 1000 * 1000, DEFAULT_BYTES_PER_PAGE, fields);
     populateFieldInfoMap(props);
 
     final Configuration dfsConfig = new Configuration();
-    final List<Footer> footers = ParquetFileReader.readFooters(dfsConfig, new Path(fileName));
+    final List<Footer> footers = ParquetFileReader.readFooters(dfsConfig, fileName);
     final Footer f = footers.iterator().next();
 
     final List<SchemaPath> columns = Lists.newArrayList();
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/store/TestAssignment.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/store/TestAssignment.java
index 51e8c1b..1297967 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/store/TestAssignment.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/store/TestAssignment.java
@@ -26,6 +26,7 @@ import org.apache.drill.exec.store.schedule.AssignmentCreator;
 import org.apache.drill.exec.store.schedule.CompleteFileWork;
 import org.apache.drill.exec.store.schedule.EndpointByteMap;
 import org.apache.drill.exec.store.schedule.EndpointByteMapImpl;
+import org.apache.hadoop.fs.Path;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -125,7 +126,7 @@ public class TestAssignment {
   private List<CompleteFileWork> generateChunks(int chunks) {
     List<CompleteFileWork> chunkList = Lists.newArrayList();
     for (int i = 0; i < chunks; i++) {
-      CompleteFileWork chunk = new CompleteFileWork(createByteMap(), 0, FILE_SIZE, "file" + i);
+      CompleteFileWork chunk = new CompleteFileWork(createByteMap(), 0, FILE_SIZE, new Path("file", Integer.toString(i)));
       chunkList.add(chunk);
     }
     return chunkList;
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/PhysicalOpUnitTestBase.java b/exec/java-exec/src/test/java/org/apache/drill/test/PhysicalOpUnitTestBase.java
index b0820e9..27a4ad6 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/PhysicalOpUnitTestBase.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/PhysicalOpUnitTestBase.java
@@ -56,6 +56,7 @@ import org.apache.drill.exec.record.VectorAccessible;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.RecordReader;
 import org.apache.drill.exec.rpc.NamedThreadFactory;
+import org.apache.hadoop.fs.Path;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
@@ -399,9 +400,9 @@ public class PhysicalOpUnitTestBase extends ExecTest {
    * @param columnsToRead
    * @return The {@link org.apache.drill.exec.store.easy.json.JSONRecordReader} corresponding to each given input path.
    */
-  public static Iterator<RecordReader> getJsonReadersFromInputFiles(DrillFileSystem fs, List<String> inputPaths, FragmentContext fragContext, List<SchemaPath> columnsToRead) {
+  public static Iterator<RecordReader> getJsonReadersFromInputFiles(DrillFileSystem fs, List<Path> inputPaths, FragmentContext fragContext, List<SchemaPath> columnsToRead) {
     List<RecordReader> readers = new ArrayList<>();
-    for (String inputPath : inputPaths) {
+    for (Path inputPath : inputPaths) {
       readers.add(new JSONRecordReader(fragContext, inputPath, fs, columnsToRead));
     }
     return readers.iterator();
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/QueryBuilder.java b/exec/java-exec/src/test/java/org/apache/drill/test/QueryBuilder.java
index b4fcedf..629714b 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/QueryBuilder.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/QueryBuilder.java
@@ -643,7 +643,7 @@ public class QueryBuilder {
    */
 
   protected String queryPlan(String columnName) throws Exception {
-    Preconditions.checkArgument(queryType == QueryType.SQL, "Can only explan an SQL query.");
+    Preconditions.checkArgument(queryType == QueryType.SQL, "Can only explain an SQL query.");
     final List<QueryDataBatch> results = results();
     final RecordBatchLoader loader = new RecordBatchLoader(client.allocator());
     final StringBuilder builder = new StringBuilder();


[drill] 01/02: DRILL-7074: Scan framework fixes and enhancements

Posted by vi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

vitalii pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git

commit 7e3b45967dbb97da18ba49a2fa6a67a48e33b092
Author: Paul Rogers <pr...@cloudera.com>
AuthorDate: Sun Mar 3 17:55:22 2019 -0800

    DRILL-7074: Scan framework fixes and enhancements
    
    Roll-up of fixes an enhancements that emerged from the effort to host the CSV reader on the new framework.
    
    closes #1676
---
 .../drill/exec/physical/impl/scan/ReaderState.java |  26 +-
 .../exec/physical/impl/scan/RowBatchReader.java    |  20 +-
 .../physical/impl/scan/ScanOperatorEvents.java     |  42 +-
 .../impl/scan/columns/ColumnsArrayParser.java      |  52 +-
 .../impl/scan/file/FileMetadataColumnsParser.java  |  71 +-
 .../impl/scan/file/FileMetadataManager.java        |  34 +
 .../physical/impl/scan/file/FileScanFramework.java |  37 +-
 .../physical/impl/scan/file/MetadataColumn.java    |  13 +
 .../impl/scan/framework/AbstractScanFramework.java |  76 ++
 .../impl/scan/framework/SchemaNegotiator.java      |   6 +-
 .../impl/scan/framework/SchemaNegotiatorImpl.java  |   6 +-
 .../impl/scan/framework/ShimBatchReader.java       |  15 +-
 .../physical/impl/scan/framework/package-info.java |  26 +-
 .../exec/physical/impl/scan/package-info.java      |  85 +-
 .../scan/project/ExplicitSchemaProjection.java     |   1 +
 .../scan/project/ReaderSchemaOrchestrator.java     | 230 +++++
 .../impl/scan/project/ScanLevelProjection.java     |  66 +-
 .../impl/scan/project/ScanSchemaOrchestrator.java  | 241 +-----
 .../impl/scan/project/SchemaLevelProjection.java   |  11 +-
 .../physical/impl/scan/project/package-info.java   |  50 ++
 .../rowSet/project/ImpliedTupleRequest.java        |   2 +-
 .../rowSet/project/RequestedTupleImpl.java         |  44 +-
 .../exec/physical/impl/scan/TestColumnsArray.java  | 172 +++-
 .../impl/scan/TestColumnsArrayFramework.java       |  19 +-
 .../physical/impl/scan/TestColumnsArrayParser.java |  46 +-
 .../impl/scan/TestConstantColumnLoader.java        | 153 ----
 .../impl/scan/TestFileMetadataColumnParser.java    | 170 +++-
 .../impl/scan/TestFileMetadataProjection.java      |   4 +-
 .../physical/impl/scan/TestFileScanFramework.java  |  11 +-
 .../physical/impl/scan/TestNullColumnLoader.java   | 281 ------
 .../physical/impl/scan/TestRowBatchMerger.java     | 459 ----------
 .../physical/impl/scan/TestScanBatchWriters.java   |   3 +
 .../impl/scan/TestScanLevelProjection.java         | 223 -----
 .../physical/impl/scan/TestScanOperatorExec.java   |  11 +-
 .../impl/scan/TestScanOrchestratorEarlySchema.java | 255 ++----
 .../impl/scan/TestScanOrchestratorLateSchema.java  |   6 +-
 .../impl/scan/TestScanOrchestratorMetadata.java    |  77 +-
 .../impl/scan/TestSchemaLevelProjection.java       | 557 ------------
 .../physical/impl/scan/TestSchemaSmoothing.java    | 946 ---------------------
 .../scan/project/TestConstantColumnLoader.java     |  45 +
 .../impl/scan/project/TestScanLevelProjection.java | 113 ++-
 .../impl/scan/project/TestSchemaSmoothing.java     | 250 +++++-
 42 files changed, 1726 insertions(+), 3229 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/ReaderState.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/ReaderState.java
index bbf12d4..b366d34 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/ReaderState.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/ReaderState.java
@@ -238,8 +238,11 @@ class ReaderState {
 
   /**
    * Prepare the schema for this reader. Called for the first reader within a
-   * scan batch, if the reader returns <tt>true</tt> from <tt>open()</tt>. If
-   * this is an early-schema reader, then the result set loader already has
+   * scan batch, if the reader returns <tt>true</tt> from <tt>open()</tt>.
+   * Asks the reader if it can provide a schema-only empty batch by calling
+   * the reader's <tt>defineSchema()</tt> method. If this is an early-schema
+   * reader, and it can provide a schema, then it should create an empty
+   * batch so that the 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 <tt>next()</tt>.
@@ -255,9 +258,10 @@ class ReaderState {
    * <li>If if turned out that the file was
    * empty when trying to read the schema, <tt>open()</tt> returned false
    * and this method should never be called.</tt>
-   * <li>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 <tt>true</tt>.
+   * <li>Otherwise, the reader does not know if it is the first reader or
+   * not. The call to <tt>defineSchema()</tt> notifies the reader that it
+   * is the first one. The reader should set up in the result set loader
+   * with an empty batch.
    * </ul>
    * <p>
    * Semantics for late-schema readers:
@@ -280,14 +284,12 @@ class ReaderState {
 
   protected boolean buildSchema() {
 
-    VectorContainer container = reader.output();
-
-    if (container != null) {
+    if (reader.defineSchema()) {
 
       // Bind the output container to the output of the scan operator.
       // This returns an empty batch with the schema filled in.
 
-      scanOp.containerAccessor.setContainer(container);
+      scanOp.containerAccessor.setContainer(reader.output());
       schemaVersion = reader.schemaVersion();
       return true;
     }
@@ -297,7 +299,8 @@ class ReaderState {
     if (! next()) {
       return false;
     }
-    container = reader.output();
+    VectorContainer container = reader.output();
+    schemaVersion = reader.schemaVersion();
     if (container.getRecordCount() == 0) {
       return true;
     }
@@ -374,8 +377,7 @@ class ReaderState {
 
   private boolean readBatch() {
 
-    // Try to read a batch. This may fail. If so, clean up the
-    // mess.
+    // Try to read a batch. This may fail. If so, clean up the mess.
 
     boolean more;
     try {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/RowBatchReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/RowBatchReader.java
index c0985b5..61de584 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/RowBatchReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/RowBatchReader.java
@@ -84,8 +84,6 @@ import org.apache.drill.exec.record.VectorContainer;
 
 public interface RowBatchReader {
 
-  enum Result { OK, LAST_BATCH, EOF }
-
   /**
    * Name used when reporting errors. Can simply be the class name.
    *
@@ -111,6 +109,22 @@ public interface RowBatchReader {
   boolean open();
 
   /**
+   * Called for the first reader within a scan. Allows the reader to
+   * provide an empty batch with only the schema filled in. Readers that
+   * are "early schema" (know the schema up front) should return true
+   * and create an empty batch. Readers that are "late schema" should
+   * return false. In that case, the scan operator will ask the reader
+   * to load an actual data batch, and infer the schema from that batch.
+   * <p>
+   * This step is optional and is purely for performance.
+   *
+   * @return true if this reader can (and has) defined an empty batch
+   * to describe the schema, false otherwise
+   */
+
+  boolean defineSchema();
+
+  /**
    * Read the next batch. Reading continues until either EOF,
    * or until the mutator indicates that the batch is full.
    * The batch is considered valid if it is non-empty. Returning
@@ -129,7 +143,7 @@ public interface RowBatchReader {
    * <tt>next()</tt> should be called again, <tt>false</tt> to indicate
    * that EOF was reached
    *
-   * @throws RutimeException (<tt>UserException</tt> preferred) if an
+   * @throws RuntimeException (<tt>UserException</tt> preferred) if an
    * error occurs that should fail the query.
    */
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/ScanOperatorEvents.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/ScanOperatorEvents.java
index 9e17414..04b2c7e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/ScanOperatorEvents.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/ScanOperatorEvents.java
@@ -20,15 +20,23 @@ package org.apache.drill.exec.physical.impl.scan;
 import org.apache.drill.exec.ops.OperatorContext;
 
 /**
- * Interface to the set of readers, and reader schema, that the
- * scan operator manages. The reader factory creates and returns
- * the readers to use for the scan, as determined by the specific
- * physical plan. The reader factory also
- * translates from the select list provided
- * in the physical plan to the actual columns returned from the
- * scan operator. The translation is reader-specific; this
- * interface allows the scan operator to trigger various
- * lifecycle events.
+ * Interface to the set of readers, and reader schema, that the scan operator
+ * manages. The reader factory creates and returns the readers to use for the
+ * scan, as determined by the specific physical plan. The reader factory also
+ * translates from the select list provided in the physical plan to the actual
+ * columns returned from the scan operator. The translation is reader-specific;
+ * this interface allows the scan operator to trigger various lifecycle events.
+ * <p>
+ * This interface decouples the scan implementation from the generic tasks
+ * needed to implement Drill's Volcano iterator protocol for operators, and
+ * Drill's schema and batch semantics. A scan implementation need only
+ * implement this interface to add plugin-specific scan behavior.
+ * <p>
+ * While this interface allows a wide variety of implementations, the intent is
+ * that most actual scanners will use the "managed" framework that handles the
+ * routine projection, vector management and other tasks that tend to be common
+ * across scanners. See {@link ScanSchemaOrchestrator} for the managed
+ * framework.
  */
 
 public interface ScanOperatorEvents {
@@ -46,11 +54,25 @@ public interface ScanOperatorEvents {
 
   void bind(OperatorContext context);
 
+  /**
+   * A scanner typically readers multiple data sources (such as files or
+   * file blocks.) A batch reader handles each read. This method returns
+   * the next reader in whatever sequence that this scan defines.
+   * <p>
+   * The preferred implementation is to create each batch reader in this
+   * call to minimize resource usage. Production queries may read
+   * thousands of files or blocks, so incremental reader creation can be
+   * far more efficient than creating readers at the start of the scan.
+   *
+   * @return a batch reader for one of the scan elements within the
+   * scan physical plan for this scan operator
+   */
+
   RowBatchReader nextReader();
 
   /**
    * Called when the scan operator itself is closed. Indicates that no more
-   * readers are available (or will be opened).
+   * readers are available.
    */
 
   void close();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/ColumnsArrayParser.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/ColumnsArrayParser.java
index f6c72b1..966a039 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/ColumnsArrayParser.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/ColumnsArrayParser.java
@@ -35,14 +35,24 @@ import org.apache.drill.exec.store.easy.text.compliant.RepeatedVarCharOutput;
  * expands to `columns`.</li>
  * <li>If the columns array appears, then no other table columns
  * can appear.</li>
- * <li>If the columns array appears, then the wildcard cannot also
- * appear, unless that wildcard expanded to be `columns` as
- * described above.</li>
+ * <li>Both 'columns' and the wildcard can appear for queries such
+ * as:<code><pre>
+ * select * from dfs.`multilevel/csv`
+ * where columns[1] < 1000</pre>
+ * </code></li>
  * <li>The query can select specific elements such as `columns`[2].
  * In this case, only array elements can appear, not the unindexed
  * `columns` column.</li>
+ * <li>If is possible for `columns` to appear twice. In this case,
+ * the project operator will make a copy.</li>
  * </ul>
  * <p>
+ * To handle these cases, the general rule is: allow any number
+ * of wildcard or `columns` appearances in the input projection, but
+ * collapse them all down to a single occurrence of `columns` in the
+ * output projection. (Upstream code will prevent `columns` from
+ * appearing twice in its non-indexed form.)
+ * <p>
  * It falls to this parser to detect a not-uncommon user error, a
  * query such as the following:<pre><code>
  * SELECT max(columns[1]) AS col1
@@ -83,7 +93,8 @@ public class ColumnsArrayParser implements ScanProjectionParser {
   @Override
   public boolean parse(RequestedColumn inCol) {
     if (requireColumnsArray && inCol.isWildcard()) {
-      expandWildcard();
+      createColumnsCol(
+          new RequestedColumnImpl(builder.rootProjection(), ColumnsArrayManager.COLUMNS_COL));
       return true;
     }
     if (! inCol.nameEquals(ColumnsArrayManager.COLUMNS_COL)) {
@@ -113,41 +124,24 @@ public class ColumnsArrayParser implements ScanProjectionParser {
           .build(logger);
       }
     }
-
-    // Special `columns` array column.
-
-    columnsArrayCol = new UnresolvedColumnsArrayColumn(inCol);
-    builder.addTableColumn(columnsArrayCol);
+    createColumnsCol(inCol);
     return true;
   }
 
-  /**
-   * Query contained SELECT *, and we know that the reader supports only
-   * the `columns` array; go ahead and expand the wildcard to the only
-   * possible column.
-   */
+  private void createColumnsCol(RequestedColumn inCol) {
+
+    // Special `columns` array column. Allow multiple, but
+    // project only one.
 
-  private void expandWildcard() {
     if (columnsArrayCol != null) {
-      throw UserException
-        .validationError()
-        .message("Cannot select columns[] and `*` together")
-        .build(logger);
+      return;
     }
-    columnsArrayCol = new UnresolvedColumnsArrayColumn(
-        new RequestedColumnImpl(builder.rootProjection(), ColumnsArrayManager.COLUMNS_COL));
+    columnsArrayCol = new UnresolvedColumnsArrayColumn(inCol);
     builder.addTableColumn(columnsArrayCol);
   }
 
   @Override
-  public void validate() {
-    if (builder.hasWildcard() && columnsArrayCol != null) {
-      throw UserException
-        .validationError()
-        .message("Cannot select `columns` and `*` together")
-        .build(logger);
-    }
-  }
+  public void validate() { }
 
   @Override
   public void validateColumn(ColumnProjection col) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileMetadataColumnsParser.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileMetadataColumnsParser.java
index f9674dc..ae8502b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileMetadataColumnsParser.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileMetadataColumnsParser.java
@@ -17,6 +17,8 @@
  */
 package org.apache.drill.exec.physical.impl.scan.file;
 
+import java.util.HashSet;
+import java.util.Set;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
@@ -38,6 +40,7 @@ public class FileMetadataColumnsParser implements ScanProjectionParser {
   private final FileMetadataManager metadataManager;
   private final Pattern partitionPattern;
   private ScanLevelProjection builder;
+  private final Set<Integer> referencedPartitions = new HashSet<>();
 
   // Output
 
@@ -64,6 +67,11 @@ public class FileMetadataColumnsParser implements ScanProjectionParser {
     if (defn != null) {
       return buildMetadataColumn(defn, inCol);
     }
+    if (inCol.isWildcard()) {
+      buildWildcard();
+
+      // Don't consider this a match.
+    }
     return false;
   }
 
@@ -80,11 +88,18 @@ public class FileMetadataColumnsParser implements ScanProjectionParser {
 
     // Partition column
 
-    builder.addMetadataColumn(
-        new PartitionColumn(
-          inCol.name(),
-          Integer.parseInt(m.group(1))));
-    hasImplicitCols = true;
+    int partitionIndex = Integer.parseInt(m.group(1));
+    if (! referencedPartitions.contains(partitionIndex)) {
+      builder.addMetadataColumn(
+          new PartitionColumn(
+            inCol.name(),
+            partitionIndex));
+
+      // Remember the partition for later wildcard expansion
+
+      referencedPartitions.add(partitionIndex);
+      hasImplicitCols = true;
+    }
     return true;
   }
 
@@ -107,8 +122,52 @@ public class FileMetadataColumnsParser implements ScanProjectionParser {
     return true;
   }
 
+  private void buildWildcard() {
+    if (metadataManager.useLegacyWildcardExpansion &&
+        metadataManager.useLegacyExpansionLocation) {
+
+      // Star column: this is a SELECT * query.
+
+      // Old-style wildcard handling inserts all partition 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.
+
+      expandPartitions();
+    }
+  }
+
   @Override
-  public void validate() { }
+  public void validate() {
+
+    // Expand partitions if using a wildcard appears, if using the
+    // feature to expand partitions for wildcards, and we want the
+    // partitions after data columns.
+
+    if (builder.hasWildcard() && metadataManager.useLegacyWildcardExpansion &&
+        ! metadataManager.useLegacyExpansionLocation) {
+      expandPartitions();
+    }
+  }
+
+  private void expandPartitions() {
+
+    // Legacy wildcard expansion: include the 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 as an array at some point.
+    // Append this after the *, keeping the * for later expansion.
+
+    for (int i = 0; i < metadataManager.partitionCount(); i++) {
+      if (referencedPartitions.contains(i)) {
+        continue;
+      }
+      builder.addMetadataColumn(new PartitionColumn(
+          metadataManager.partitionName(i), i));
+      referencedPartitions.add(i);
+    }
+    hasImplicitCols = true;
+  }
 
   @Override
   public void validateColumn(ColumnProjection outCol) { }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileMetadataManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileMetadataManager.java
index fe4332a..ba49a9f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileMetadataManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileMetadataManager.java
@@ -53,6 +53,31 @@ public class FileMetadataManager implements MetadataManager, SchemaProjectionRes
   protected final String partitionDesignator;
   protected List<FileMetadataColumnDefn> implicitColDefns = new ArrayList<>();
   protected Map<String, FileMetadataColumnDefn> fileMetadataColIndex = CaseInsensitiveMap.newHashMap();
+
+  /**
+   * Indicates whether to expand partition columns when the query contains a wildcard.
+   * Supports queries such as the following:<code><pre>
+   * select * from dfs.`partitioned-dir`
+   * </pre><code>
+   * In which the output columns will be (columns, dir0) if the partitioned directory
+   * has one level of nesting.
+   *
+   * See {@link TestImplicitFileColumns#testImplicitColumns}
+   */
+  protected final boolean useLegacyWildcardExpansion;
+
+  /**
+   * In legacy mode, above, Drill expands partition columns whenever the
+   * wildcard appears. Drill 1.1 - 1.11 put expanded partition columns after
+   * data columns. This is actually a better position as it minimizes changes
+   * the row layout for files at different depths. Drill 1.12 moved them before
+   * data columns: at the location of the wildcard.
+   * <p>
+   * This flag, when set, uses the Drill 1.12 position. Later enhancements
+   * can unset this flag to go back to the future: use the preferred location
+   * after other columns.
+   */
+  protected final boolean useLegacyExpansionLocation;
   private final FileMetadataColumnsParser parser;
 
   // Internal state
@@ -84,7 +109,11 @@ public class FileMetadataManager implements MetadataManager, SchemaProjectionRes
    */
 
   public FileMetadataManager(OptionSet optionManager,
+      boolean useLegacyWildcardExpansion,
+      boolean useLegacyExpansionLocation,
       Path rootDir, List<Path> files) {
+    this.useLegacyWildcardExpansion = useLegacyWildcardExpansion;
+    this.useLegacyExpansionLocation = useLegacyExpansionLocation;
     scanRootDir = rootDir;
 
     partitionDesignator = optionManager.getString(ExecConstants.FILESYSTEM_PARTITION_COLUMN_LABEL);
@@ -117,6 +146,11 @@ public class FileMetadataManager implements MetadataManager, SchemaProjectionRes
     }
   }
 
+  public FileMetadataManager(OptionSet optionManager,
+      Path rootDir, List<Path> files) {
+    this(optionManager, false, false, rootDir, files);
+  }
+
   private int computeMaxPartition(List<Path> files) {
     int maxLen = 0;
     for (Path filePath : files) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileScanFramework.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileScanFramework.java
index 609e9f0..6ecf0cf 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileScanFramework.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileScanFramework.java
@@ -32,19 +32,46 @@ import org.apache.hadoop.mapred.FileSplit;
 
 /**
  * The file scan framework adds into the scan framework support for implicit
- * file metadata columns.
+ * file metadata columns. The file scan framework brings together a number of
+ * components:
+ * <ul>
+ * <li>The projection list provided by the physical operator definition. This
+ * list identifies the set of "output" columns whih this framework is obliged
+ * to produce.</li>
+ * <li>The set of files and/or blocks to read.</li>
+ * <li>The file system configuration to use for working with the files
+ * or blocks.</li>
+ * <li>The factory class to create a reader for each of the files or blocks
+ * defined above. (Readers are created one-by-one as files are read.)</li>
+ * <li>Options as defined by the base class.</li>
+ * </ul>
+ * <p>
+ * @See {AbstractScanFramework} for details.
  */
 
 public class FileScanFramework extends BaseFileScanFramework<FileSchemaNegotiator> {
 
-  public interface FileReaderCreator {
+  /**
+   * Creates a batch reader on demand. Unlike earlier versions of Drill,
+   * this framework creates readers one by one, when they are needed.
+   * Doing so avoids excessive resource demands that come from creating
+   * potentially thousands of readers up front.
+   * <p>
+   * The reader itself is unique to each file type. This interface
+   * provides a common interface that this framework can use to create the
+   * file-specific reader on demand.
+   */
+
+  public interface FileReaderFactory {
     ManagedReader<FileSchemaNegotiator> makeBatchReader(
         DrillFileSystem dfs,
         FileSplit split) throws ExecutionSetupException;
   }
 
   /**
-   * Implementation of the file-level schema negotiator.
+   * Implementation of the file-level schema negotiator. At present, no
+   * file-specific features exist. This class shows, however, where we would
+   * add such features.
    */
 
   public static class FileSchemaNegotiatorImpl extends SchemaNegotiatorImpl
@@ -55,12 +82,12 @@ public class FileScanFramework extends BaseFileScanFramework<FileSchemaNegotiato
     }
   }
 
-  private final FileReaderCreator readerCreator;
+  private final FileReaderFactory readerCreator;
 
   public FileScanFramework(List<SchemaPath> projection,
       List<? extends FileWork> files,
       Configuration fsConf,
-      FileReaderCreator readerCreator) {
+      FileReaderFactory readerCreator) {
     super(projection, files, fsConf);
     this.readerCreator = readerCreator;
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/MetadataColumn.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/MetadataColumn.java
index 4a15ff7..bf13265 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/MetadataColumn.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/MetadataColumn.java
@@ -44,4 +44,17 @@ public abstract class MetadataColumn extends ResolvedColumn implements ConstantC
   public String name() { return schema.getName(); }
 
   public abstract MetadataColumn resolve(FileMetadata fileInfo, VectorSource source, int sourceIndex);
+
+  @Override
+  public String toString() {
+    return new StringBuilder()
+        .append("[")
+        .append(getClass().getSimpleName())
+        .append(" schema=\"")
+        .append(schema.toString())
+        .append(", value=")
+        .append(value)
+        .append("]")
+        .toString();
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/AbstractScanFramework.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/AbstractScanFramework.java
index 5407901..d285261 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/AbstractScanFramework.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/AbstractScanFramework.java
@@ -40,15 +40,91 @@ import org.apache.drill.exec.physical.impl.scan.project.ScanSchemaOrchestrator;
  * config options, and implements the matching "managed reader". All details
  * of setup, projection, and so on are handled by the framework and the components
  * that the framework builds upon.
+ *
+ * <h4>Inputs</h4>
+ *
+ * At this basic level, a scan framework requires just a few simple inputs:
+ * <ul>
+ * <li>The projection list provided by the physical operator definition. This
+ * list identifies the set of "output" columns whih this framework is obliged
+ * to produce.</li>
+ * <li>The operator context which provides access to a memory allocator and
+ * other plumbing items.</li>
+ * <li>A method to create a reader for each of the files or blocks
+ * defined above. (Readers are created one-by-one as files are read.)</li>
+ * <li>The data type to use for projected columns which the reader cannot
+ * provide. (Drill allows such columns and fills in null values: traditionally
+ * nullable Int, but customizable here.)
+ * <li>Various other options.</li>
+ * </ul>
+ *
+ * <h4>Orchestration</h4>
+ *
+ * The above is sufficient to drive the entire scan operator functionality.
+ * Projection is done generically and is the same for all files. Only the
+ * reader (created via the factory class) differs from one type of file to
+ * another.
+ * <p>
+ * The framework achieves the work described below= by composing a large
+ * set of detailed classes, each of which performs some specific task. This
+ * structure leaves the reader to simply infer schema and read data.
+ * <p>
+ * In particular, rather than do all the orchestration here (which would tie
+ * that logic to the scan operation), the detailed work is delegated to the
+ * {@link ScanSchemaOrchestrator} class, with this class as a "shim" between
+ * the the Scan events API and the schema orchestrator implementation.
+ *
+ * <h4>Reader Integration</h4>
+ *
+ * The details of how a file is structured, how a schema is inferred, how
+ * data is decoded: all that is encapsulated in the reader. The only real
+ * Interaction between the reader and the framework is:
+ * <ul>
+ * <li>The reader "negotiates" a schema with the framework. The framework
+ * knows the projection list from the query plan, knows something about
+ * data types (whether a column should be scalar, a map or an array), and
+ * knows about the schema already defined by prior readers. The reader knows
+ * what schema it can produce (if "early schema.") The schema negotiator
+ * class handles this task.</li>
+ * <li>The reader reads data from the file and populates value vectors a
+ * batch at a time. The framework creates the result set loader to use for
+ * this work. The schema negotiator returns that loader to the reader, which
+ * uses it during read.
+ * <p>
+ * It is important to note that the result set loader also defines a schema:
+ * the schema requested by the reader. If the reader wants to read three
+ * columns, a, b, and c, then that is the schema that the result set loader
+ * supports. This is true even if the query plan only wants column a, or
+ * wants columns c, a. The framework handles the projection task so the
+ * reader does not have to worry about it. Reading an unwanted column
+ * is low cost: the result set loader will have provided a "dummy" column
+ * writer that simply discards the value. This is just as fast as having the
+ * reader use if-statements or a table to determine which columns to save.
+ * <p>
+ * A reader may be "late schema", true "schema on read." In this case, the
+ * reader simply tells the result set loader to create a new column reader
+ * on the fly. The framework will work out if that new column is to be
+ * projected and will return either a real column writer (projected column)
+ * or a dummy column writer (unprojected column.)</li>
+ * <li>The reader then reads batches of data until all data is read. The
+ * result set loader signals when a batch is full; the reader should not
+ * worry about this detail itself.</li>
+ * <li>The reader then releases its resources.</li>
+ * </ul>
  */
 
 public abstract class AbstractScanFramework<T extends SchemaNegotiator> implements ScanOperatorEvents {
 
+  // Inputs
+
   protected final List<SchemaPath> projection;
   protected MajorType nullType;
   protected int maxBatchRowCount;
   protected int maxBatchByteCount;
   protected OperatorContext context;
+
+  // Internal state
+
   protected ScanSchemaOrchestrator scanOrchestrator;
 
   public AbstractScanFramework(List<SchemaPath> projection) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/SchemaNegotiator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/SchemaNegotiator.java
index 6318812..dead9cb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/SchemaNegotiator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/SchemaNegotiator.java
@@ -65,9 +65,13 @@ public interface SchemaNegotiator {
    * columns during the read.
    *
    * @param schema the table schema if known at open time
+   * @param isComplete true if the schema is complete: if it can be used
+   * to define an empty schema-only batch for the first reader. Set to
+   * false if the schema is partial: if the reader must read rows to
+   * determine the full schema
    */
 
-  void setTableSchema(TupleMetadata schema);
+  void setTableSchema(TupleMetadata schema, boolean isComplete);
 
   /**
    * Set the preferred batch size (which may be overridden by the
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/SchemaNegotiatorImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/SchemaNegotiatorImpl.java
index 46f363d..0841049 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/SchemaNegotiatorImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/SchemaNegotiatorImpl.java
@@ -53,6 +53,7 @@ public class SchemaNegotiatorImpl implements SchemaNegotiator {
   protected final AbstractScanFramework<?> basicFramework;
   private final ShimBatchReader<? extends SchemaNegotiator> shim;
   protected TupleMetadata tableSchema;
+  protected boolean isSchemaComplete;
   protected int batchSize = ValueVector.MAX_ROW_COUNT;
 
   public SchemaNegotiatorImpl(AbstractScanFramework<?> framework, ShimBatchReader<? extends SchemaNegotiator> shim) {
@@ -66,8 +67,9 @@ public class SchemaNegotiatorImpl implements SchemaNegotiator {
   }
 
   @Override
-  public void setTableSchema(TupleMetadata schema) {
+  public void setTableSchema(TupleMetadata schema, boolean isComplete) {
     tableSchema = schema;
+    this.isSchemaComplete = schema != null && isComplete;
   }
 
   @Override
@@ -97,4 +99,6 @@ public class SchemaNegotiatorImpl implements SchemaNegotiator {
   public boolean isProjectionEmpty() {
     return basicFramework.scanOrchestrator().isProjectNone();
   }
+
+  public boolean isSchemaComplete() { return isSchemaComplete; }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/ShimBatchReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/ShimBatchReader.java
index 0dc3c57..a97b329 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/ShimBatchReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/ShimBatchReader.java
@@ -19,7 +19,7 @@ package org.apache.drill.exec.physical.impl.scan.framework;
 
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.physical.impl.scan.RowBatchReader;
-import org.apache.drill.exec.physical.impl.scan.project.ScanSchemaOrchestrator.ReaderSchemaOrchestrator;
+import org.apache.drill.exec.physical.impl.scan.project.ReaderSchemaOrchestrator;
 import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
 import org.apache.drill.exec.record.VectorContainer;
 
@@ -44,6 +44,7 @@ public class ShimBatchReader<T extends SchemaNegotiator> implements RowBatchRead
   protected final AbstractScanFramework<T> manager;
   protected final ManagedReader<T> reader;
   protected final ReaderSchemaOrchestrator readerOrchestrator;
+  protected SchemaNegotiatorImpl schemaNegotiator;
   protected ResultSetLoader tableLoader;
 
   /**
@@ -96,10 +97,19 @@ public class ShimBatchReader<T extends SchemaNegotiator> implements RowBatchRead
   }
 
   @Override
+  public boolean defineSchema() {
+    if (schemaNegotiator.isSchemaComplete()) {
+      readerOrchestrator.defineSchema();
+      return true;
+    }
+    return false;
+  }
+
+  @Override
   public boolean next() {
 
     // The reader may report EOF, but the result set loader might
-    // have a lookhead row.
+    // have a lookahead row.
 
     if (eof && ! tableLoader.hasRows()) {
       return false;
@@ -181,6 +191,7 @@ public class ShimBatchReader<T extends SchemaNegotiator> implements RowBatchRead
   }
 
   public ResultSetLoader build(SchemaNegotiatorImpl schemaNegotiator) {
+    this.schemaNegotiator = schemaNegotiator;
     readerOrchestrator.setBatchSize(schemaNegotiator.batchSize);
     tableLoader = readerOrchestrator.makeTableLoader(schemaNegotiator.tableSchema);
     return tableLoader;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/package-info.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/package-info.java
index efd881b..096b844 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/package-info.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/package-info.java
@@ -29,7 +29,7 @@
  * is an old version without a new column c, while file B includes the column.
  * And so on.
  * <p>
- * The scan operator here works to ensure schema continuity as much as
+ * The scan operator works to ensure schema continuity as much as
  * possible, smoothing out "soft" schema changes that are simply artifacts of
  * reading a collection of files. Only "hard" changes (true changes) are
  * passed downstream.
@@ -157,5 +157,29 @@
  * output batch in the order specified by the original SELECT list (or table order,
  * if the original SELECT had a wildcard.) Fortunately, this is just involves
  * moving around pointers to vectors; no actual data is moved during projection.
+ *
+ * <h4>Class Structure</h4>
+ *
+ * Some of the key classes here include:
+ * <ul>
+ * <li>{@link RowBatchReader} an extremely simple interface for reading data.
+ * We would like many developers to create new plugins and readers. The simplified
+ * interface pushes all complexity into the scan framework, leaving the reader to
+ * just read.</li>
+ * <li>{@link ShimBatchReader} an implementation of the above that converts from
+ * the simplified API to add additional structure to work with the result set loader.
+ * (The base interface is agnostic about how rows are read.)</li>
+ * <li>{@link ScheamNegotiator} and interface that allows a batch reader to
+ * "negotiate" a schema with the scan framework. The scan framework knows the
+ * columns that are to be projected. The reader knows what columns it can offer.
+ * The schema negotiator works out how to combine the two. It expresses the result
+ * as a result set loader. Column writers are defined for all columns that the
+ * reader wants to read, but only the materialized (projected) columns have actual
+ * vectors behind them. The non-projected columns are "free-wheeling" "dummy"
+ * writers.
+ * </li>
+ *
+ * And, yes, sorry for the terminology. File "readers" read from files, but
+ * use column "writers" to write to value vectors.
  */
 package org.apache.drill.exec.physical.impl.scan.framework;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/package-info.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/package-info.java
index 3e302a1..d7de30a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/package-info.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/package-info.java
@@ -36,9 +36,90 @@
  * <p>
  * See {@link ScanOperatorExec} for details of the scan operator protocol
  * and components.
+ *
+ * <h4>Traditional Class Structure<h4>
+ * The original design was simple: but required each reader to handle many
+ * detailed tasks.
+ * <pre><code>
+ *  +------------+          +-----------+
+ *  | Scan Batch |    +---> | ScanBatch |
+ *  |  Creator   |    |     +-----------+
+ *  +------------+    |           |
+ *         |          |           |
+ *         v          |           |
+ *  +------------+    |           v
+ *  |   Format   | ---+   +---------------+
+ *  |   Plugin   | -----> | Record Reader |
+ *  +------------+        +---------------+
+ *
+ * </code></pre>
+ *
+ * The scan batch creator is unique to each storage plugin and is created
+ * based on the physical operator configuration ("pop config"). The
+ * scan batch creator delegates to the format plugin to create both the
+ * scan batch (the scan operator) and the set of readers which the scan
+ * batch will manage.
+ * <p>
+ * The scan batch
+ * provides a <code>Mutator</code> that creates the vectors used by the
+ * record readers. Schema continuity comes from reusing the Mutator from one
+ * file/block to the next.
+ * <p>
+ * One characteristic of this system is that all the record readers are
+ * created up front. If we must read 1000 blocks, we'll create 1000 record
+ * readers. Developers must be very careful to only allocate resources when
+ * the reader is opened, and release resources when the reader is closed.
+ * Else, resource bloat becomes a large problem.
+ *
+ * <h4>Revised Class Structure</h4>
+ *
+ * The new design is more complex because it divides tasks up into separate
+ * classes. The class structure is larger, but each class is smaller, more
+ * focused and does just one task.
+ * <pre><code>
+ *  +------------+          +---------------+
+ *  | Scan Batch | -------> | Format Plugin |
+ *  |  Creator   |          +---------------+
+ *  +------------+          /        |       \
+ *                         /         |        \
+ *    +---------------------+        |         \ +---------------+
+ *    | OperatorRecordBatch |        |     +---->| ScanFramework |
+ *    +---------------------+        |     |     +---------------+
+ *                                   v     |            |
+ *                         +------------------+         |
+ *                         | ScanOperatorExec |         |
+ *                         +------------------+         v
+ *                                   |            +--------------+
+ *                                   +----------> | Batch Reader |
+ *                                                +--------------+
+ * </code></pre>
+ *
+ * Here, the scan batch creator again delegates to the format plugin. The
+ * format plugin creates three objects:
+ * <ul>
+ * <li>The <code>OperatorRecordBatch</code>, which encapsulates the Volcano
+ * iterator protocol. It also holds onto the output batch. This allows the
+ * operator implementation to just focus on its specific job.</li>
+ * <li>The <code>ScanOperatorExec</code> is the operator implementation for
+ * the new result-set-loader based scan.</li>
+ * <li>The scan framework is specific to each kind of reader. It handles
+ * everything which is unique to that reader. Rather than inheriting from
+ * the scan itself, the framework follows the strategy pattern: it says how
+ * to do a scan for the target format.<li>
+ * </ul>
+ *
+ * The overall structure uses the "composition" pattern: what is combined
+ * into a small set of classes in the traditional model is broken out into
+ * focused classes in the revised model.
+ * <p>
+ * A key part of the scan strategy is the batch reader. ("Batch" because
+ * it reads an entire batch at a time, using the result set loader.) The
+ * framework creates batch readers one by one as needed. Resource bloat
+ * is less of an issue because only one batch reader instance exists at
+ * any time for each scan operator instance.
  * <p>
- * See the "managed" package for a reusable framework for handling the
- * details of batches, schema and so on.
+ * Each of the above is further broken down into additional classes to
+ * handle projection and so on.
  */
 
 package org.apache.drill.exec.physical.impl.scan;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ExplicitSchemaProjection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ExplicitSchemaProjection.java
index 41cc595..c0bcfa3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ExplicitSchemaProjection.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ExplicitSchemaProjection.java
@@ -41,6 +41,7 @@ import org.apache.drill.exec.record.metadata.TupleMetadata;
  */
 
 public class ExplicitSchemaProjection extends SchemaLevelProjection {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExplicitSchemaProjection.class);
 
   public ExplicitSchemaProjection(ScanLevelProjection scanProj,
       TupleMetadata tableSchema,
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ReaderSchemaOrchestrator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ReaderSchemaOrchestrator.java
new file mode 100644
index 0000000..029b6a0
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ReaderSchemaOrchestrator.java
@@ -0,0 +1,230 @@
+/*
+ * 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.project;
+
+import org.apache.drill.exec.physical.impl.scan.project.ResolvedTuple.ResolvedRow;
+import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
+import org.apache.drill.exec.physical.rowSet.impl.OptionBuilder;
+import org.apache.drill.exec.physical.rowSet.impl.ResultSetLoaderImpl;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.ValueVector;
+
+/**
+ * Orchestrates projection tasks for a single reader within the set that the
+ * scan operator manages. Vectors are reused across readers, but via a vector
+ * cache. All other state is distinct between readers.
+ */
+
+public class ReaderSchemaOrchestrator implements VectorSource {
+
+  private final ScanSchemaOrchestrator scanOrchestrator;
+  private int readerBatchSize;
+  private ResultSetLoaderImpl tableLoader;
+  private int prevTableSchemaVersion = -1;
+
+  /**
+   * Assembles the table, metadata and null columns into the final output
+   * batch to be sent downstream. The key goal of this class is to "smooth"
+   * schema changes in this output batch by absorbing trivial schema changes
+   * that occur across readers.
+   */
+
+  private ResolvedRow rootTuple;
+  private VectorContainer tableContainer;
+
+  public ReaderSchemaOrchestrator(ScanSchemaOrchestrator scanSchemaOrchestrator) {
+    scanOrchestrator = scanSchemaOrchestrator;
+    readerBatchSize = scanOrchestrator.scanBatchRecordLimit;
+  }
+
+  public void setBatchSize(int size) {
+    if (size > 0) {
+      readerBatchSize = Math.min(size, scanOrchestrator.scanBatchRecordLimit);
+    }
+  }
+
+  public ResultSetLoader makeTableLoader(TupleMetadata tableSchema) {
+    OptionBuilder options = new OptionBuilder();
+    options.setRowCountLimit(readerBatchSize);
+    options.setVectorCache(scanOrchestrator.vectorCache);
+    options.setBatchSizeLimit(scanOrchestrator.scanBatchByteLimit);
+
+    // Set up a selection list if available and is a subset of
+    // table columns. (Only needed for non-wildcard queries.)
+    // The projection list includes all candidate table columns
+    // whether or not they exist in the up-front schema. Handles
+    // the odd case where the reader claims a fixed schema, but
+    // adds a column later.
+
+    if (! scanOrchestrator.scanProj.projectAll()) {
+      options.setProjectionSet(scanOrchestrator.scanProj.readerProjection());
+    }
+    options.setSchema(tableSchema);
+
+    // Create the table loader
+
+    tableLoader = new ResultSetLoaderImpl(scanOrchestrator.allocator, options.build());
+    return tableLoader;
+  }
+
+  public boolean hasSchema() {
+    return prevTableSchemaVersion >= 0;
+  }
+
+  public void defineSchema() {
+    tableLoader.startEmptyBatch();
+    endBatch();
+  }
+
+  public void startBatch() {
+    tableLoader.startBatch();
+  }
+
+  /**
+   * Build the final output batch by projecting columns from the three input sources
+   * to the output batch. First, build the metadata and/or null columns for the
+   * table row count. Then, merge the sources.
+   */
+
+  public void endBatch() {
+
+    // Get the batch results in a container.
+
+    tableContainer = tableLoader.harvest();
+
+    // If the schema changed, set up the final projection based on
+    // the new (or first) schema.
+
+    if (prevTableSchemaVersion < tableLoader.schemaVersion()) {
+      reviseOutputProjection();
+    } else {
+
+      // Fill in the null and metadata columns.
+
+      populateNonDataColumns();
+    }
+    rootTuple.setRowCount(tableContainer.getRecordCount());
+  }
+
+  private void populateNonDataColumns() {
+    int rowCount = tableContainer.getRecordCount();
+    scanOrchestrator.metadataManager.load(rowCount);
+    rootTuple.loadNulls(rowCount);
+  }
+
+  /**
+   * Create the list of null columns by comparing the SELECT list against the
+   * columns available in the batch schema. Create null columns for those that
+   * are missing. This is done for the first batch, and any time the schema
+   * changes. (For early-schema, the projection occurs once as the schema is set
+   * up-front and does not change.) For a SELECT *, the null column check
+   * only need be done if null columns were created when mapping from a prior
+   * schema.
+   */
+
+  private void reviseOutputProjection() {
+
+    // Do the table-schema level projection; the final matching
+    // of projected columns to available columns.
+
+    TupleMetadata tableSchema = tableLoader.harvestSchema();
+    if (scanOrchestrator.schemaSmoother != null) {
+      doSmoothedProjection(tableSchema);
+    } else if (scanOrchestrator.scanProj.hasWildcard()) {
+      doWildcardProjection(tableSchema);
+    } else {
+      doExplicitProjection(tableSchema);
+    }
+
+    // Combine metadata, nulls and batch data to form the final
+    // output container. Columns are created by the metadata and null
+    // loaders only in response to a batch, so create the first batch.
+
+    rootTuple.buildNulls(scanOrchestrator.vectorCache);
+    scanOrchestrator.metadataManager.define();
+    populateNonDataColumns();
+    rootTuple.project(tableContainer, scanOrchestrator.outputContainer);
+    prevTableSchemaVersion = tableLoader.schemaVersion();
+  }
+
+  private void doSmoothedProjection(TupleMetadata tableSchema) {
+    rootTuple = new ResolvedRow(
+        new NullColumnBuilder(scanOrchestrator.nullType, scanOrchestrator.allowRequiredNullColumns));
+    scanOrchestrator.schemaSmoother.resolve(tableSchema, rootTuple);
+  }
+
+  /**
+   * Query contains a wildcard. The schema-level projection includes
+   * all columns provided by the reader.
+   */
+
+  private void doWildcardProjection(TupleMetadata tableSchema) {
+    rootTuple = new ResolvedRow(null);
+    new WildcardSchemaProjection(scanOrchestrator.scanProj,
+        tableSchema, rootTuple, scanOrchestrator.schemaResolvers);
+  }
+
+  /**
+   * Explicit projection: include only those columns actually
+   * requested by the query, which may mean filling in null
+   * columns for projected columns that don't actually exist
+   * in the table.
+   *
+   * @param tableSchema newly arrived schema
+   */
+
+  private void doExplicitProjection(TupleMetadata tableSchema) {
+    rootTuple = new ResolvedRow(
+        new NullColumnBuilder(scanOrchestrator.nullType, scanOrchestrator.allowRequiredNullColumns));
+    new ExplicitSchemaProjection(scanOrchestrator.scanProj,
+            tableSchema, rootTuple,
+            scanOrchestrator.schemaResolvers);
+  }
+
+  @Override
+  public ValueVector vector(int index) {
+    return tableContainer.getValueVector(index).getValueVector();
+  }
+
+  public void close() {
+    RuntimeException ex = null;
+    try {
+      if (tableLoader != null) {
+        tableLoader.close();
+        tableLoader = null;
+      }
+    }
+    catch (RuntimeException e) {
+      ex = e;
+    }
+    try {
+      if (rootTuple != null) {
+        rootTuple.close();
+        rootTuple = null;
+      }
+    }
+    catch (RuntimeException e) {
+      ex = ex == null ? e : ex;
+    }
+    scanOrchestrator.metadataManager.endFile();
+    if (ex != null) {
+      throw ex;
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ScanLevelProjection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ScanLevelProjection.java
index 83d40a3..f90f722 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ScanLevelProjection.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ScanLevelProjection.java
@@ -100,8 +100,6 @@ import org.apache.drill.exec.physical.rowSet.project.RequestedTupleImpl;
 
 public class ScanLevelProjection {
 
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScanLevelProjection.class);
-
   /**
    * Interface for add-on parsers, avoids the need to create
    * a single, tightly-coupled parser for all types of columns.
@@ -128,12 +126,26 @@ public class ScanLevelProjection {
 
   // Internal state
 
+  protected boolean includesWildcard;
   protected boolean sawWildcard;
 
   // Output
 
   protected List<ColumnProjection> outputCols = new ArrayList<>();
+
+  /**
+   * Projection definition for the scan a whole. Parsed form of the input
+   * projection list.
+   */
+
   protected RequestedTuple outputProjection;
+
+  /**
+   * Projection definition passed to each reader. This is the set of
+   * columns that the reader is asked to provide.
+   */
+
+  protected RequestedTuple readerProjection;
   protected boolean hasWildcard;
   protected boolean emptyProjection = true;
 
@@ -158,6 +170,18 @@ public class ScanLevelProjection {
     for (ScanProjectionParser parser : parsers) {
       parser.bind(this);
     }
+
+    // First pass: check if a wildcard exists.
+
+    for (RequestedColumn inCol : outputProjection.projections()) {
+      if (inCol.isWildcard()) {
+        includesWildcard = true;
+        break;
+      }
+    }
+
+    // Second pass: process remaining columns.
+
     for (RequestedColumn inCol : outputProjection.projections()) {
       if (inCol.isWildcard()) {
         mapWildcard(inCol);
@@ -169,6 +193,23 @@ public class ScanLevelProjection {
     for (ScanProjectionParser parser : parsers) {
       parser.build();
     }
+
+    // Create the reader projection which includes either all columns
+    // (saw a wildcard) or just the unresolved columns (which excludes
+    // implicit columns.)
+
+    List<RequestedColumn> outputProj;
+    if (hasWildcard()) {
+      outputProj = null;
+    } else {
+      outputProj = new ArrayList<>();
+      for (ColumnProjection col : outputCols) {
+        if (col instanceof UnresolvedColumn) {
+          outputProj.add(((UnresolvedColumn) col).element());
+        }
+      }
+    }
+    readerProjection = RequestedTupleImpl.build(outputProj);
   }
 
   /**
@@ -181,6 +222,7 @@ public class ScanLevelProjection {
 
     // Wildcard column: this is a SELECT * query.
 
+    assert includesWildcard;
     if (sawWildcard) {
       throw new IllegalArgumentException("Duplicate * entry in project list");
     }
@@ -245,6 +287,15 @@ public class ScanLevelProjection {
       }
     }
 
+    // If the project list has a wildcard, and the column is not one recognized
+    // by the specialized parsers above, then just ignore it. It is likely a duplicate
+    // column name. In any event, it will be processed by the Project operator on
+    // top of this scan.
+
+    if (includesWildcard) {
+      return;
+    }
+
     // This is a desired table column.
 
     addTableColumn(
@@ -281,15 +332,6 @@ public class ScanLevelProjection {
       for (ScanProjectionParser parser : parsers) {
         parser.validateColumn(outCol);
       }
-      switch (outCol.nodeType()) {
-      case UnresolvedColumn.UNRESOLVED:
-        if (hasWildcard()) {
-          throw new IllegalArgumentException("Cannot select table columns and * together");
-        }
-        break;
-      default:
-        break;
-      }
     }
   }
 
@@ -333,6 +375,8 @@ public class ScanLevelProjection {
 
   public RequestedTuple rootProjection() { return outputProjection; }
 
+  public RequestedTuple readerProjection() { return readerProjection; }
+
   @Override
   public String toString() {
     return new StringBuilder()
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ScanSchemaOrchestrator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ScanSchemaOrchestrator.java
index fe78f5a..a5d6ca2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ScanSchemaOrchestrator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ScanSchemaOrchestrator.java
@@ -23,15 +23,10 @@ import java.util.List;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.physical.impl.scan.project.ResolvedTuple.ResolvedRow;
 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.rowSet.ResultSetLoader;
-import org.apache.drill.exec.physical.rowSet.impl.OptionBuilder;
-import org.apache.drill.exec.physical.rowSet.impl.ResultSetLoaderImpl;
 import org.apache.drill.exec.physical.rowSet.impl.ResultVectorCacheImpl;
 import org.apache.drill.exec.record.VectorContainer;
-import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.vector.ValueVector;
 
 /**
@@ -154,212 +149,6 @@ public class ScanSchemaOrchestrator {
   public static final int DEFAULT_BATCH_BYTE_COUNT = ValueVector.MAX_BUFFER_SIZE;
   public static final int MAX_BATCH_ROW_COUNT = ValueVector.MAX_ROW_COUNT;
 
-  /**
-   * Orchestrates projection tasks for a single reader with the set that the
-   * scan operator manages. Vectors are reused across readers, but via a vector
-   * cache. All other state is distinct between readers.
-   */
-
-  public class ReaderSchemaOrchestrator implements VectorSource {
-
-    private int readerBatchSize;
-    private ResultSetLoaderImpl tableLoader;
-    private int prevTableSchemaVersion = -1;
-
-    /**
-     * Assembles the table, metadata and null columns into the final output
-     * batch to be sent downstream. The key goal of this class is to "smooth"
-     * schema changes in this output batch by absorbing trivial schema changes
-     * that occur across readers.
-     */
-
-    private ResolvedRow rootTuple;
-    private VectorContainer tableContainer;
-
-    public ReaderSchemaOrchestrator() {
-      readerBatchSize = scanBatchRecordLimit;
-    }
-
-    public void setBatchSize(int size) {
-      if (size > 0) {
-        readerBatchSize = Math.min(size, scanBatchRecordLimit);
-      }
-    }
-
-    public ResultSetLoader makeTableLoader(TupleMetadata tableSchema) {
-      OptionBuilder options = new OptionBuilder();
-      options.setRowCountLimit(readerBatchSize);
-      options.setVectorCache(vectorCache);
-      options.setBatchSizeLimit(scanBatchByteLimit);
-
-      // Set up a selection list if available and is a subset of
-      // table columns. (Only needed for non-wildcard queries.)
-      // The projection list includes all candidate table columns
-      // whether or not they exist in the up-front schema. Handles
-      // the odd case where the reader claims a fixed schema, but
-      // adds a column later.
-
-      if (! scanProj.projectAll()) {
-        options.setProjectionSet(scanProj.rootProjection());
-      }
-      options.setSchema(tableSchema);
-
-      // Create the table loader
-
-      tableLoader = new ResultSetLoaderImpl(allocator, options.build());
-
-      // If a schema is given, create a zero-row batch to announce the
-      // schema downstream in the form of an empty batch.
-
-      if (tableSchema != null) {
-        tableLoader.startEmptyBatch();
-        endBatch();
-      }
-
-      return tableLoader;
-    }
-
-    public boolean hasSchema() {
-      return prevTableSchemaVersion >= 0;
-    }
-
-    public void startBatch() {
-      tableLoader.startBatch();
-    }
-
-    /**
-     * Build the final output batch by projecting columns from the three input sources
-     * to the output batch. First, build the metadata and/or null columns for the
-     * table row count. Then, merge the sources.
-     */
-
-    public void endBatch() {
-
-      // Get the batch results in a container.
-
-      tableContainer = tableLoader.harvest();
-
-      // If the schema changed, set up the final projection based on
-      // the new (or first) schema.
-
-      if (prevTableSchemaVersion < tableLoader.schemaVersion()) {
-        reviseOutputProjection();
-      } else {
-
-        // Fill in the null and metadata columns.
-
-        populateNonDataColumns();
-      }
-      rootTuple.setRowCount(tableContainer.getRecordCount());
-    }
-
-    private void populateNonDataColumns() {
-      int rowCount = tableContainer.getRecordCount();
-      metadataManager.load(rowCount);
-      rootTuple.loadNulls(rowCount);
-    }
-
-    /**
-     * Create the list of null columns by comparing the SELECT list against the
-     * columns available in the batch schema. Create null columns for those that
-     * are missing. This is done for the first batch, and any time the schema
-     * changes. (For early-schema, the projection occurs once as the schema is set
-     * up-front and does not change.) For a SELECT *, the null column check
-     * only need be done if null columns were created when mapping from a prior
-     * schema.
-     */
-
-    private void reviseOutputProjection() {
-
-      // Do the table-schema level projection; the final matching
-      // of projected columns to available columns.
-
-      TupleMetadata tableSchema = tableLoader.harvestSchema();
-      if (schemaSmoother != null) {
-        doSmoothedProjection(tableSchema);
-      } else if (scanProj.hasWildcard()) {
-        doWildcardProjection(tableSchema);
-      } else {
-        doExplicitProjection(tableSchema);
-      }
-
-      // Combine metadata, nulls and batch data to form the final
-      // output container. Columns are created by the metadata and null
-      // loaders only in response to a batch, so create the first batch.
-
-      rootTuple.buildNulls(vectorCache);
-      metadataManager.define();
-      populateNonDataColumns();
-      rootTuple.project(tableContainer, outputContainer);
-      prevTableSchemaVersion = tableLoader.schemaVersion();
-    }
-
-    private void doSmoothedProjection(TupleMetadata tableSchema) {
-      rootTuple = new ResolvedRow(
-          new NullColumnBuilder(nullType, allowRequiredNullColumns));
-      schemaSmoother.resolve(tableSchema, rootTuple);
-    }
-
-    /**
-     * Query contains a wildcard. The schema-level projection includes
-     * all columns provided by the reader.
-     */
-
-    private void doWildcardProjection(TupleMetadata tableSchema) {
-      rootTuple = new ResolvedRow(null);
-      new WildcardSchemaProjection(scanProj,
-          tableSchema, rootTuple, schemaResolvers);
-    }
-
-    /**
-     * Explicit projection: include only those columns actually
-     * requested by the query, which may mean filling in null
-     * columns for projected columns that don't actually exist
-     * in the table.
-     *
-     * @param tableSchema newly arrived schema
-     */
-
-    private void doExplicitProjection(TupleMetadata tableSchema) {
-      rootTuple = new ResolvedRow(
-          new NullColumnBuilder(nullType, allowRequiredNullColumns));
-      new ExplicitSchemaProjection(scanProj,
-              tableSchema, rootTuple,
-              schemaResolvers);
-    }
-
-    @Override
-    public ValueVector vector(int index) {
-      return tableContainer.getValueVector(index).getValueVector();
-    }
-
-    public void close() {
-      RuntimeException ex = null;
-      try {
-        if (tableLoader != null) {
-          tableLoader.close();
-          tableLoader = null;
-        }
-      }
-      catch (RuntimeException e) {
-        ex = e;
-      }
-      try {
-        if (rootTuple != null) {
-          rootTuple.close();
-          rootTuple = null;
-        }
-      }
-      catch (RuntimeException e) {
-        ex = ex == null ? e : ex;
-      }
-      metadataManager.endFile();
-      if (ex != null) {
-        throw ex;
-      }
-    }
-  }
-
   // Configuration
 
   /**
@@ -367,16 +156,16 @@ public class ScanSchemaOrchestrator {
    * not set, the null type is the Drill default.
    */
 
-  private MajorType nullType;
+  MajorType nullType;
 
   /**
    * Creates the metadata (file and directory) columns, if needed.
    */
 
-  private MetadataManager metadataManager;
-  private final BufferAllocator allocator;
-  private int scanBatchRecordLimit = DEFAULT_BATCH_ROW_COUNT;
-  private int scanBatchByteLimit = DEFAULT_BATCH_BYTE_COUNT;
+  MetadataManager metadataManager;
+  final BufferAllocator allocator;
+  int scanBatchRecordLimit = DEFAULT_BATCH_ROW_COUNT;
+  int scanBatchByteLimit = DEFAULT_BATCH_BYTE_COUNT;
   private final List<ScanProjectionParser> parsers = new ArrayList<>();
 
   /**
@@ -389,7 +178,7 @@ public class ScanSchemaOrchestrator {
   List<SchemaProjectionResolver> schemaResolvers = new ArrayList<>();
 
   private boolean useSchemaSmoothing;
-  private boolean allowRequiredNullColumns;
+  boolean allowRequiredNullColumns;
 
   // Internal state
 
@@ -402,14 +191,14 @@ public class ScanSchemaOrchestrator {
    * vectors rather than vector instances, this cache can be deprecated.
    */
 
-  private ResultVectorCacheImpl vectorCache;
-  private ScanLevelProjection scanProj;
+  ResultVectorCacheImpl vectorCache;
+  ScanLevelProjection scanProj;
   private ReaderSchemaOrchestrator currentReader;
-  private SchemaSmoother schemaSmoother;
+  SchemaSmoother schemaSmoother;
 
   // Output
 
-  private VectorContainer outputContainer;
+  VectorContainer outputContainer;
 
   public ScanSchemaOrchestrator(BufferAllocator allocator) {
     this.allocator = allocator;
@@ -493,20 +282,12 @@ public class ScanSchemaOrchestrator {
 
     ScanProjectionParser parser = metadataManager.projectionParser();
     if (parser != null) {
-
-      // For compatibility with Drill 1.12, insert the file metadata
-      // parser before others so that, in a wildcard query, metadata
-      // columns appear before others (such as the `columns` column.)
-      // This is temporary and should be removed once the test framework
-      // is restored to Drill 1.11 functionality.
-
       parsers.add(parser);
     }
 
     // Parse the projection list.
 
     scanProj = new ScanLevelProjection(projection, parsers);
-
     if (scanProj.hasWildcard() && useSchemaSmoothing) {
       schemaSmoother = new SchemaSmoother(scanProj, schemaResolvers);
     }
@@ -526,7 +307,7 @@ public class ScanSchemaOrchestrator {
 
   public ReaderSchemaOrchestrator startReader() {
     closeReader();
-    currentReader = new ReaderSchemaOrchestrator();
+    currentReader = new ReaderSchemaOrchestrator(this);
     return currentReader;
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/SchemaLevelProjection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/SchemaLevelProjection.java
index c7bae27..a756114 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/SchemaLevelProjection.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/SchemaLevelProjection.java
@@ -61,8 +61,6 @@ import org.apache.drill.exec.record.metadata.TupleMetadata;
 
 public class SchemaLevelProjection {
 
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SchemaLevelProjection.class);
-
   /**
    * Schema-level projection is customizable. Implement this interface, and
    * add an instance to the scan orchestrator, to perform custom mappings
@@ -81,10 +79,7 @@ public class SchemaLevelProjection {
 
   protected SchemaLevelProjection(
         List<SchemaProjectionResolver> resolvers) {
-    if (resolvers == null) {
-      resolvers = new ArrayList<>();
-    }
-    this.resolvers = resolvers;
+    this.resolvers = resolvers == null ? new ArrayList<>() : resolvers;
     for (SchemaProjectionResolver resolver : resolvers) {
       resolver.startResolution();
     }
@@ -97,6 +92,8 @@ public class SchemaLevelProjection {
         return;
       }
     }
-    throw new IllegalStateException("No resolver for column: " + col.nodeType());
+    throw new IllegalStateException(
+        String.format("No resolver for column `%s` of type %d",
+            col.name(), col.nodeType()));
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/package-info.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/package-info.java
index a5a52c0..155fcf8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/package-info.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/package-info.java
@@ -22,6 +22,56 @@
  * or may be "missing" with null values applied. The code here prepares
  * a run-time projection plan based on the actual table schema.
  * <p>
+ * Looks at schema as a set of transforms.
+ * <ul>
+ * <li>Scan-level projection list from the query plan: The list of columns
+ * (or the wildcard) as requested by the user in the query. The planner
+ * determines which columns to project. In Drill, projection is speculative:
+ * it is a list of names which the planner hopes will appear in the data
+ * files. The reader must make up columns (the infamous nullable INT) when
+ * it turns out that no such column exists. Else, the reader must figure out
+ * the data type for any columns that does exist.
+ * <p>
+ * The scan project list defines the set of columns which the scan operator
+ * is obliged to send downstream. Ideally, the scan operator sends exactly the
+ * same schema (the project list with types filled in) for all batches. Since
+ * batches may come from different files, the scan operator is obligated to
+ * unify the schemas from those files (or blocks.)</ul>
+ * <li>Reader (file)-level projection occurs for each reader. A single scan
+ * may use multiple readers to read data. Each reader may offer more information
+ * about the schema. For example, a Parquet reader can obtain schema information
+ * from the Parquet headers. A JDBC reader obtains schema information from the
+ * returned schema. This is called "early schema." File-based readers can at least
+ * add implicit file or partition columns.
+ * <p>
+ * The result is a refined schema: the scan level schema with more information
+ * filled in. For Parquet, all projection information can be filled in. For
+ * CSV or JSON, we can only add file metadata information, but not yet the
+ * actual data schema.</ul>
+ * <li>Batch-level schema: once a reader reads actual data, it now knows
+ * exactly what it read. This is the "schema on read model." Thus, after reading
+ * a batch, any remaining uncertainty about the projected schema is removed.
+ * The actual data defined data types and so on.
+ * <p>
+ * Readers such as JSON and CSV are "late schema": they don't know the data
+ * schema until they read the file. This is true "schema on read." Further, for
+ * JSON, the data may change from one batch to the next as the reader "discovers"
+ * fields that did not appear in earlier batches. This requires some amount of
+ * "schema smoothing": the ability to preserve a consistent output schema even
+ * as the input schema jiggles around some.</ul>
+ * </ul>
+ * <p>
+ * The goal of this mechanism is to handle the above use cases cleanly, in a
+ * common set of classes, and to avoid the need for each reader to figure out
+ * all these issues for themselves (as was the case with earlier versions of
+ * Drill.)
+ * <p>
+ * Because these issues are complex, the code itself is complex. To make the
+ * code easier to manage, each bit of functionality is encapsulated in a
+ * distinct class. Classes combine via composition to create a "framework"
+ * suitable for each kind of reader: whether it be early or late schema,
+ * file-based or something else, etc.
+ * <p>
  * The core concept is one of successive refinement of the project
  * list through a set of rewrites:
  * <ul>
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/project/ImpliedTupleRequest.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/project/ImpliedTupleRequest.java
index c1e383e..f464bae 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/project/ImpliedTupleRequest.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/project/ImpliedTupleRequest.java
@@ -37,7 +37,7 @@ public class ImpliedTupleRequest implements RequestedTuple {
       new ImpliedTupleRequest(false);
   public static final List<RequestedColumn> EMPTY_COLS = new ArrayList<>();
 
-  private boolean allProjected;
+  private final boolean allProjected;
 
   public ImpliedTupleRequest(boolean allProjected) {
     this.allProjected = allProjected;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/project/RequestedTupleImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/project/RequestedTupleImpl.java
index cd782c7..4643c57 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/project/RequestedTupleImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/project/RequestedTupleImpl.java
@@ -73,7 +73,7 @@ import org.apache.drill.exec.record.metadata.TupleNameSpace;
 
 public class RequestedTupleImpl implements RequestedTuple {
 
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RequestedTupleImpl.class);
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RequestedTupleImpl.class);
 
   private final RequestedColumnImpl parent;
   private final TupleNameSpace<RequestedColumn> projection = new TupleNameSpace<>();
@@ -86,6 +86,13 @@ public class RequestedTupleImpl implements RequestedTuple {
     this.parent = parent;
   }
 
+  public RequestedTupleImpl(List<RequestedColumn> cols) {
+    parent = null;
+    for (RequestedColumn col : cols) {
+      projection.add(col.name(), col);
+    }
+  }
+
   @Override
   public RequestedColumn get(String colName) {
     return projection.get(colName.toLowerCase());
@@ -119,10 +126,43 @@ public class RequestedTupleImpl implements RequestedTuple {
   }
 
   /**
+   * Create a requested tuple projection from a rewritten top-level
+   * projection list. The columns within the list have already been parsed to
+   * pick out arrays, maps and scalars. The list must not include the
+   * wildcard: a wildcard list must be passed in as a null list. An
+   * empty list means project nothing. Null list means project all, else
+   * project only the columns in the list.
+   *
+   * @param projList top-level, parsed columns
+   * @return the tuple projection for the top-leel row
+   */
+
+  public static RequestedTuple build(List<RequestedColumn> projList) {
+    if (projList == null) {
+      return new ImpliedTupleRequest(true);
+    }
+    if (projList.isEmpty()) {
+      return new ImpliedTupleRequest(false);
+    }
+    return new RequestedTupleImpl(projList);
+  }
+
+  /**
    * Parse a projection list. The list should consist of a list of column names;
-   * any wildcards should have been processed by the caller. An empty list means
+   * or wildcards. An empty list means
    * nothing is projected. A null list means everything is projected (that is, a
    * null list here is equivalent to a wildcard in the SELECT statement.)
+   * <p>
+   * The projection list may include both a wildcard and column names (as in
+   * the case of implicit columns.) This results in a final list that both
+   * says that everything is projected, and provides the list of columns.
+   * <p>
+   * Parsing is used at two different times. First, to parse the list from
+   * the physical operator. This has the case above: an explicit wildcard
+   * and/or additional columns. Then, this class is used again to prepare the
+   * physical projection used when reading. In this case, wildcards should
+   * be removed, implicit columns pulled out, and just the list of read-level
+   * columns should remain.
    *
    * @param projList
    *          the list of projected columns, or null if no projection is to be
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestColumnsArray.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestColumnsArray.java
index 3587e27..88ccd3c 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestColumnsArray.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestColumnsArray.java
@@ -22,33 +22,42 @@ import static org.junit.Assert.fail;
 
 import java.util.List;
 
+import org.apache.drill.categories.RowSetTests;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.physical.impl.scan.columns.ColumnsArrayManager;
 import org.apache.drill.exec.physical.impl.scan.file.FileMetadataManager;
+import org.apache.drill.exec.physical.impl.scan.project.ReaderSchemaOrchestrator;
 import org.apache.drill.exec.physical.impl.scan.project.ScanSchemaOrchestrator;
-import org.apache.drill.exec.physical.impl.scan.project.ScanSchemaOrchestrator.ReaderSchemaOrchestrator;
 import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
 import org.apache.drill.exec.physical.rowSet.impl.RowSetTestUtils;
 import org.apache.drill.exec.record.metadata.SchemaBuilder;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.test.SubOperatorTest;
 import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
-import org.apache.drill.test.rowSet.RowSetComparison;
+import org.apache.drill.test.rowSet.RowSetUtilities;
 import org.apache.hadoop.fs.Path;
 import org.junit.Test;
-
+import org.junit.experimental.categories.Category;
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
 
+/**
+ * Test the "columns" array mechanism integrated with the scan schema
+ * orchestrator including simulating reading data.
+ */
+
+@Category(RowSetTests.class)
 public class TestColumnsArray extends SubOperatorTest {
 
-  /**
-   * Test columns array. The table must be able to support it by having a
-   * matching column.
-   */
+  private static class MockScanner {
+    ScanSchemaOrchestrator scanner;
+    ReaderSchemaOrchestrator reader;
+    ResultSetLoader loader;
+  }
 
-  @Test
-  public void testColumnsArray() {
+  private MockScanner buildScanner(List<SchemaPath> projList) {
+
+    MockScanner mock = new MockScanner();
 
     // Set up the file metadata manager
 
@@ -64,21 +73,19 @@ public class TestColumnsArray extends SubOperatorTest {
 
     // Configure the schema orchestrator
 
-    ScanSchemaOrchestrator scanner = new ScanSchemaOrchestrator(fixture.allocator());
-    scanner.withMetadata(metadataManager);
-    scanner.addParser(colsManager.projectionParser());
-    scanner.addResolver(colsManager.resolver());
+    mock.scanner = new ScanSchemaOrchestrator(fixture.allocator());
+    mock.scanner.withMetadata(metadataManager);
+    mock.scanner.addParser(colsManager.projectionParser());
+    mock.scanner.addResolver(colsManager.resolver());
 
-    // SELECT filename, columns, dir0 ...
+    // SELECT <proj list> ...
 
-    scanner.build(RowSetTestUtils.projectList(ScanTestUtils.FILE_NAME_COL,
-        ColumnsArrayManager.COLUMNS_COL,
-        ScanTestUtils.partitionColName(0)));
+    mock.scanner.build(projList);
 
     // FROM z.csv
 
     metadataManager.startFile(filePath);
-    ReaderSchemaOrchestrator reader = scanner.startReader();
+    mock.reader = mock.scanner.startReader();
 
     // Table schema (columns: VARCHAR[])
 
@@ -86,7 +93,25 @@ public class TestColumnsArray extends SubOperatorTest {
         .addArray(ColumnsArrayManager.COLUMNS_COL, MinorType.VARCHAR)
         .buildSchema();
 
-    ResultSetLoader loader = reader.makeTableLoader(tableSchema);
+    mock.loader = mock.reader.makeTableLoader(tableSchema);
+
+    // First empty batch
+
+    mock.reader.defineSchema();
+    return mock;
+  }
+
+  /**
+   * Test columns array. The table must be able to support it by having a
+   * matching column.
+   */
+
+  @Test
+  public void testColumnsArray() {
+
+    MockScanner mock = buildScanner(RowSetTestUtils.projectList(ScanTestUtils.FILE_NAME_COL,
+        ColumnsArrayManager.COLUMNS_COL,
+        ScanTestUtils.partitionColName(0)));
 
     // Verify empty batch.
 
@@ -99,18 +124,18 @@ public class TestColumnsArray extends SubOperatorTest {
       SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
          .build();
 
-      assertNotNull(scanner.output());
-      new RowSetComparison(expected)
-         .verifyAndClearAll(fixture.wrap(scanner.output()));
+      assertNotNull(mock.scanner.output());
+      RowSetUtilities.verify(expected,
+         fixture.wrap(mock.scanner.output()));
     }
 
     // Create a batch of data.
 
-    reader.startBatch();
-    loader.writer()
+    mock.reader.startBatch();
+    mock.loader.writer()
       .addRow(new Object[] {new String[] {"fred", "flintstone"}})
       .addRow(new Object[] {new String[] {"barney", "rubble"}});
-    reader.endBatch();
+    mock. reader.endBatch();
 
     // Verify
 
@@ -120,11 +145,100 @@ public class TestColumnsArray extends SubOperatorTest {
         .addRow("z.csv", new String[] {"barney", "rubble"}, "x")
         .build();
 
-      new RowSetComparison(expected)
-          .verifyAndClearAll(fixture.wrap(scanner.output()));
+      RowSetUtilities.verify(expected,
+          fixture.wrap(mock.scanner.output()));
     }
 
-    scanner.close();
+    mock.scanner.close();
+  }
+
+  @Test
+  public void testWildcard() {
+
+    MockScanner mock = buildScanner(RowSetTestUtils.projectAll());
+
+    // Verify empty batch.
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addArray("columns", MinorType.VARCHAR)
+        .buildSchema();
+    {
+      SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
+         .build();
+
+      assertNotNull(mock.scanner.output());
+      RowSetUtilities.verify(expected,
+         fixture.wrap(mock.scanner.output()));
+    }
+
+    // Create a batch of data.
+
+    mock.reader.startBatch();
+    mock.loader.writer()
+      .addRow(new Object[] {new String[] {"fred", "flintstone"}})
+      .addRow(new Object[] {new String[] {"barney", "rubble"}});
+    mock. reader.endBatch();
+
+    // Verify
+
+    {
+      SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addSingleCol(new String[] {"fred", "flintstone"})
+        .addSingleCol(new String[] {"barney", "rubble"})
+        .build();
+
+      RowSetUtilities.verify(expected,
+          fixture.wrap(mock.scanner.output()));
+    }
+
+    mock.scanner.close();
+  }
+
+  @Test
+  public void testWildcardAndFileMetadata() {
+
+    MockScanner mock = buildScanner(RowSetTestUtils.projectList(
+        ScanTestUtils.FILE_NAME_COL,
+        SchemaPath.DYNAMIC_STAR,
+        ScanTestUtils.partitionColName(0)));
+
+    // Verify empty batch.
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .add("filename", MinorType.VARCHAR)
+        .addArray("columns", MinorType.VARCHAR)
+        .addNullable("dir0", MinorType.VARCHAR)
+        .buildSchema();
+    {
+      SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
+         .build();
+
+      assertNotNull(mock.scanner.output());
+      RowSetUtilities.verify(expected,
+         fixture.wrap(mock.scanner.output()));
+    }
+
+    // Create a batch of data.
+
+    mock.reader.startBatch();
+    mock.loader.writer()
+      .addRow(new Object[] {new String[] {"fred", "flintstone"}})
+      .addRow(new Object[] {new String[] {"barney", "rubble"}});
+    mock. reader.endBatch();
+
+    // Verify
+
+    {
+      SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addRow("z.csv", new String[] {"fred", "flintstone"}, "x")
+        .addRow("z.csv", new String[] {"barney", "rubble"}, "x")
+        .build();
+
+      RowSetUtilities.verify(expected,
+          fixture.wrap(mock.scanner.output()));
+    }
+
+    mock.scanner.close();
   }
 
   private ScanSchemaOrchestrator buildScan(List<SchemaPath> cols) {
@@ -160,6 +274,7 @@ public class TestColumnsArray extends SubOperatorTest {
     try {
       ReaderSchemaOrchestrator reader = scanner.startReader();
       reader.makeTableLoader(tableSchema);
+      reader.defineSchema();
       fail();
     } catch (IllegalStateException e) {
       // Expected
@@ -180,6 +295,7 @@ public class TestColumnsArray extends SubOperatorTest {
     try {
       ReaderSchemaOrchestrator reader = scanner.startReader();
       reader.makeTableLoader(tableSchema);
+      reader.defineSchema();
       fail();
     } catch (IllegalStateException e) {
       // Expected
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestColumnsArrayFramework.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestColumnsArrayFramework.java
index 4f32b56..e7a0188 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestColumnsArrayFramework.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestColumnsArrayFramework.java
@@ -27,6 +27,7 @@ import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 
+import org.apache.drill.categories.RowSetTests;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos.MinorType;
@@ -46,7 +47,7 @@ import org.apache.drill.test.SubOperatorTest;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.FileSplit;
 import org.junit.Test;
-
+import org.junit.experimental.categories.Category;
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
 
 
@@ -54,6 +55,7 @@ import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
  * Test the columns-array specific behavior in the columns scan framework.
  */
 
+@Category(RowSetTests.class)
 public class TestColumnsArrayFramework extends SubOperatorTest {
 
   private static final Path MOCK_FILE_PATH = new Path("file:/w/x/y/z.csv");
@@ -101,7 +103,7 @@ public class TestColumnsArrayFramework extends SubOperatorTest {
     @Override
     public boolean open(ColumnsSchemaNegotiator negotiator) {
       this.negotiator = negotiator;
-      negotiator.setTableSchema(schema);
+      negotiator.setTableSchema(schema, true);
       negotiator.build();
       return true;
     }
@@ -115,6 +117,11 @@ public class TestColumnsArrayFramework extends SubOperatorTest {
     public void close() { }
   }
 
+  /**
+   * Test including a column other than "columns". Occurs when
+   * using implicit columns.
+   */
+
   @Test
   public void testNonColumnsProjection() {
 
@@ -143,6 +150,10 @@ public class TestColumnsArrayFramework extends SubOperatorTest {
     scanFixture.close();
   }
 
+  /**
+   * Test projecting just the `columns` column.
+   */
+
   @Test
   public void testColumnsProjection() {
 
@@ -171,6 +182,10 @@ public class TestColumnsArrayFramework extends SubOperatorTest {
     scanFixture.close();
   }
 
+  /**
+   * Test including a specific index of `columns` such as
+   * `columns`[1].
+   */
   @Test
   public void testColumnsIndexProjection() {
 
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestColumnsArrayParser.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestColumnsArrayParser.java
index e2c8a21..d1e91a2 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestColumnsArrayParser.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestColumnsArrayParser.java
@@ -23,6 +23,7 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import org.apache.drill.categories.RowSetTests;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.physical.impl.scan.columns.ColumnsArrayManager;
@@ -35,9 +36,10 @@ import org.apache.drill.exec.physical.rowSet.impl.RowSetTestUtils;
 import org.apache.drill.test.SubOperatorTest;
 import org.apache.hadoop.fs.Path;
 import org.junit.Test;
-
+import org.junit.experimental.categories.Category;
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
 
+@Category(RowSetTests.class)
 public class TestColumnsArrayParser extends SubOperatorTest {
 
   /**
@@ -255,5 +257,45 @@ public class TestColumnsArrayParser extends SubOperatorTest {
     assertEquals(FileMetadataColumn.ID, scanProj.columns().get(2).nodeType());
   }
 
-  // TODO: Test Columns element projection
+  /**
+   * If a query is of the form:
+   * <pre><code>
+   * select * from dfs.`multilevel/csv` where columns[1] < 1000
+   * </code><pre>
+   * Then the projection list passed to the scan operator
+   * includes both the wildcard and the `columns` array.
+   * We can ignore one of them.
+   */
+
+  @Test
+  public void testWildcardAndColumns() {
+    ScanLevelProjection scanProj = new ScanLevelProjection(
+        RowSetTestUtils.projectList(
+            SchemaPath.DYNAMIC_STAR,
+            ColumnsArrayManager.COLUMNS_COL),
+        ScanTestUtils.parsers(new ColumnsArrayParser(true)));
+
+    assertFalse(scanProj.projectAll());
+    assertEquals(2, scanProj.requestedCols().size());
+
+    assertEquals(1, scanProj.columns().size());
+    assertEquals(ColumnsArrayManager.COLUMNS_COL, scanProj.columns().get(0).name());
+
+    // Verify column type
+
+    assertEquals(UnresolvedColumnsArrayColumn.ID, scanProj.columns().get(0).nodeType());
+  }
+
+  @Test
+  public void testColumnsAsMap() {
+    try {
+        new ScanLevelProjection(
+          RowSetTestUtils.projectList("columns.x"),
+          ScanTestUtils.parsers(new ColumnsArrayParser(true)));
+        fail();
+    }
+    catch (UserException e) {
+      // Expected
+    }
+  }
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestConstantColumnLoader.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestConstantColumnLoader.java
deleted file mode 100644
index 330a661..0000000
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestConstantColumnLoader.java
+++ /dev/null
@@ -1,153 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.physical.impl.scan;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.drill.common.types.TypeProtos.DataMode;
-import org.apache.drill.common.types.TypeProtos.MajorType;
-import org.apache.drill.common.types.TypeProtos.MinorType;
-import org.apache.drill.exec.physical.impl.scan.file.FileMetadata;
-import org.apache.drill.exec.physical.impl.scan.file.FileMetadataColumn;
-import org.apache.drill.exec.physical.impl.scan.file.FileMetadataColumnDefn;
-import org.apache.drill.exec.physical.impl.scan.file.PartitionColumn;
-import org.apache.drill.exec.physical.impl.scan.project.ConstantColumnLoader;
-import org.apache.drill.exec.physical.impl.scan.project.ConstantColumnLoader.ConstantColumnSpec;
-import org.apache.drill.exec.physical.rowSet.impl.ResultVectorCacheImpl;
-import org.apache.drill.exec.record.BatchSchema;
-import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.record.metadata.SchemaBuilder;
-import org.apache.drill.exec.store.ColumnExplorer.ImplicitFileColumns;
-import org.apache.drill.test.SubOperatorTest;
-import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
-import org.apache.drill.test.rowSet.RowSetComparison;
-import org.apache.hadoop.fs.Path;
-import org.junit.Test;
-
-public class TestConstantColumnLoader extends SubOperatorTest {
-
-  private static class DummyColumn implements ConstantColumnSpec {
-
-    private String name;
-    private MaterializedField schema;
-    private String value;
-
-    public DummyColumn(String name, MajorType type, String value) {
-      this.name = name;
-      this.schema = MaterializedField.create(name, type);
-      this.value = value;
-    }
-
-    @Override
-    public String name() { return name; }
-
-    @Override
-    public MaterializedField schema() { return schema; }
-
-    @Override
-    public String value() { return value; }
-  }
-
-  /**
-   * Test the static column loader using one column of each type.
-   * The null column is of type int, but the associated value is of
-   * type string. This is a bit odd, but works out because we detect that
-   * the string value is null and call setNull on the writer, and avoid
-   * using the actual data.
-   */
-
-  @Test
-  public void testConstantColumnLoader() {
-
-    MajorType aType = MajorType.newBuilder()
-        .setMinorType(MinorType.VARCHAR)
-        .setMode(DataMode.REQUIRED)
-        .build();
-    MajorType bType = MajorType.newBuilder()
-        .setMinorType(MinorType.VARCHAR)
-        .setMode(DataMode.OPTIONAL)
-        .build();
-
-    List<ConstantColumnSpec> defns = new ArrayList<>();
-    defns.add(
-        new DummyColumn("a", aType, "a-value" ));
-    defns.add(
-        new DummyColumn("b", bType, "b-value" ));
-
-    ResultVectorCacheImpl cache = new ResultVectorCacheImpl(fixture.allocator());
-    ConstantColumnLoader staticLoader = new ConstantColumnLoader(cache, defns);
-
-    // Create a batch
-
-    staticLoader.load(2);
-
-    // Verify
-
-    BatchSchema expectedSchema = new SchemaBuilder()
-        .add("a", aType)
-        .add("b", bType)
-        .build();
-    SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
-        .addRow("a-value", "b-value")
-        .addRow("a-value", "b-value")
-        .build();
-
-    new RowSetComparison(expected)
-        .verifyAndClearAll(fixture.wrap(staticLoader.load(2)));
-    staticLoader.close();
-  }
-
-  @Test
-  public void testFileMetadata() {
-
-    FileMetadata fileInfo = new FileMetadata(new Path("hdfs:///w/x/y/z.csv"), new Path("hdfs:///w"));
-    List<ConstantColumnSpec> defns = new ArrayList<>();
-    FileMetadataColumnDefn iDefn = new FileMetadataColumnDefn(
-        ScanTestUtils.SUFFIX_COL, ImplicitFileColumns.SUFFIX);
-    FileMetadataColumn iCol = new FileMetadataColumn(ScanTestUtils.SUFFIX_COL,
-        iDefn, fileInfo, null, 0);
-    defns.add(iCol);
-
-    String partColName = ScanTestUtils.partitionColName(1);
-    PartitionColumn pCol = new PartitionColumn(partColName, 1, fileInfo, null, 0);
-    defns.add(pCol);
-
-    ResultVectorCacheImpl cache = new ResultVectorCacheImpl(fixture.allocator());
-    ConstantColumnLoader staticLoader = new ConstantColumnLoader(cache, defns);
-
-    // Create a batch
-
-    staticLoader.load(2);
-
-    // Verify
-
-    BatchSchema expectedSchema = new SchemaBuilder()
-        .add(ScanTestUtils.SUFFIX_COL, MinorType.VARCHAR)
-        .addNullable(partColName, MinorType.VARCHAR)
-        .build();
-    SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
-        .addRow("csv", "y")
-        .addRow("csv", "y")
-        .build();
-
-    new RowSetComparison(expected)
-        .verifyAndClearAll(fixture.wrap(staticLoader.load(2)));
-    staticLoader.close();
-  }
-}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestFileMetadataColumnParser.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestFileMetadataColumnParser.java
index 08aeed5..a6de5e6 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestFileMetadataColumnParser.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestFileMetadataColumnParser.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import java.util.List;
 
+import org.apache.drill.categories.RowSetTests;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.physical.impl.scan.file.FileMetadataColumn;
 import org.apache.drill.exec.physical.impl.scan.file.FileMetadataManager;
@@ -33,9 +34,10 @@ import org.apache.drill.exec.physical.rowSet.impl.RowSetTestUtils;
 import org.apache.drill.test.SubOperatorTest;
 import org.apache.hadoop.fs.Path;
 import org.junit.Test;
-
+import org.junit.experimental.categories.Category;
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
 
+@Category(RowSetTests.class)
 public class TestFileMetadataColumnParser extends SubOperatorTest {
 
   @Test
@@ -135,8 +137,12 @@ public class TestFileMetadataColumnParser extends SubOperatorTest {
     assertEquals(PartitionColumn.ID, scanProj.columns().get(0).nodeType());
   }
 
+  /**
+   * Test wildcard expansion.
+   */
+
   @Test
-  public void testWildcard() {
+  public void testRevisedWildcard() {
     Path filePath = new Path("hdfs:///w/x/y/z.csv");
     FileMetadataManager metadataManager = new FileMetadataManager(
         fixture.getOptionManager(),
@@ -153,15 +159,45 @@ public class TestFileMetadataColumnParser extends SubOperatorTest {
   }
 
   /**
+   * Legacy (prior version) wildcard expansion always expands partition
+   * columns.
+   */
+
+  @Test
+  public void testLegacyWildcard() {
+    Path filePath = new Path("hdfs:///w/x/y/z.csv");
+    FileMetadataManager metadataManager = new FileMetadataManager(
+        fixture.getOptionManager(),
+        true, // Use legacy wildcard expansion
+        true, // Put partitions at end
+        new Path("hdfs:///w"),
+        Lists.newArrayList(filePath));
+
+    ScanLevelProjection scanProj = new ScanLevelProjection(
+        RowSetTestUtils.projectAll(),
+        Lists.newArrayList(metadataManager.projectionParser()));
+
+    List<ColumnProjection> cols = scanProj.columns();
+    assertEquals(3, cols.size());
+    assertEquals(UnresolvedColumn.WILDCARD, cols.get(0).nodeType());
+    assertEquals(PartitionColumn.ID, cols.get(1).nodeType());
+    assertEquals(0, ((PartitionColumn) cols.get(1)).partition());
+    assertEquals(PartitionColumn.ID, cols.get(2).nodeType());
+    assertEquals(1, ((PartitionColumn) cols.get(2)).partition());
+  }
+
+  /**
    * Combine wildcard and file metadata columms. The wildcard expands
    * table columns but not metadata columns.
    */
 
   @Test
-  public void testWildcardAndFileMetadata() {
+  public void testLegacyWildcardAndFileMetadata() {
     Path filePath = new Path("hdfs:///w/x/y/z.csv");
     FileMetadataManager metadataManager = new FileMetadataManager(
         fixture.getOptionManager(),
+        true, // Use legacy wildcard expansion
+        false, // Put partitions at end
         new Path("hdfs:///w"),
         Lists.newArrayList(filePath));
 
@@ -173,10 +209,12 @@ public class TestFileMetadataColumnParser extends SubOperatorTest {
         Lists.newArrayList(metadataManager.projectionParser()));
 
     List<ColumnProjection> cols = scanProj.columns();
-    assertEquals(3, cols.size());
+    assertEquals(5, cols.size());
     assertEquals(UnresolvedColumn.WILDCARD, cols.get(0).nodeType());
     assertEquals(FileMetadataColumn.ID, cols.get(1).nodeType());
     assertEquals(FileMetadataColumn.ID, cols.get(2).nodeType());
+    assertEquals(PartitionColumn.ID, cols.get(3).nodeType());
+    assertEquals(PartitionColumn.ID, cols.get(4).nodeType());
   }
 
   /**
@@ -185,10 +223,12 @@ public class TestFileMetadataColumnParser extends SubOperatorTest {
    */
 
   @Test
-  public void testWildcardAndFileMetadataMixed() {
+  public void testLegacyWildcardAndFileMetadataMixed() {
     Path filePath = new Path("hdfs:///w/x/y/z.csv");
     FileMetadataManager metadataManager = new FileMetadataManager(
         fixture.getOptionManager(),
+        true, // Use legacy wildcard expansion
+        false, // Put partitions at end
         new Path("hdfs:///w"),
         Lists.newArrayList(filePath));
 
@@ -200,18 +240,25 @@ public class TestFileMetadataColumnParser extends SubOperatorTest {
         Lists.newArrayList(metadataManager.projectionParser()));
 
     List<ColumnProjection> cols = scanProj.columns();
-    assertEquals(3, cols.size());
+    assertEquals(5, cols.size());
     assertEquals(FileMetadataColumn.ID, cols.get(0).nodeType());
     assertEquals(UnresolvedColumn.WILDCARD, cols.get(1).nodeType());
     assertEquals(FileMetadataColumn.ID, cols.get(2).nodeType());
+    assertEquals(PartitionColumn.ID, cols.get(3).nodeType());
+    assertEquals(PartitionColumn.ID, cols.get(4).nodeType());
   }
 
   /**
-   * Include both a wildcard and a partition column.
+   * Include both a wildcard and a partition column. The wildcard, in
+   * legacy mode, will create partition columns for any partitions not
+   * mentioned in the project list.
+   * <p>
+   * Tests proposed functionality: included only requested partition
+   * columns.
    */
 
   @Test
-  public void testWildcardAndPartition() {
+  public void testRevisedWildcardAndPartition() {
     Path filePath = new Path("hdfs:///w/x/y/z.csv");
     FileMetadataManager metadataManager = new FileMetadataManager(
         fixture.getOptionManager(),
@@ -229,6 +276,113 @@ public class TestFileMetadataColumnParser extends SubOperatorTest {
       assertEquals(PartitionColumn.ID, cols.get(1).nodeType());
   }
 
+  @Test
+  public void testLegacyWildcardAndPartition() {
+    Path filePath = new Path("hdfs:///w/x/y/z.csv");
+    FileMetadataManager metadataManager = new FileMetadataManager(
+        fixture.getOptionManager(),
+        true, // Use legacy wildcard expansion
+        true, // Put partitions at end
+        new Path("hdfs:///w"),
+        Lists.newArrayList(filePath));
+
+    ScanLevelProjection scanProj = new ScanLevelProjection(
+        RowSetTestUtils.projectList(SchemaPath.DYNAMIC_STAR,
+            ScanTestUtils.partitionColName(8)),
+        Lists.newArrayList(metadataManager.projectionParser()));
+
+      List<ColumnProjection> cols = scanProj.columns();
+      assertEquals(4, cols.size());
+      assertEquals(UnresolvedColumn.WILDCARD, cols.get(0).nodeType());
+      assertEquals(PartitionColumn.ID, cols.get(1).nodeType());
+      assertEquals(0, ((PartitionColumn) cols.get(1)).partition());
+      assertEquals(PartitionColumn.ID, cols.get(2).nodeType());
+      assertEquals(1, ((PartitionColumn) cols.get(2)).partition());
+      assertEquals(PartitionColumn.ID, cols.get(3).nodeType());
+      assertEquals(8, ((PartitionColumn) cols.get(3)).partition());
+  }
+
+  @Test
+  public void testPreferredPartitionExpansion() {
+    Path filePath = new Path("hdfs:///w/x/y/z.csv");
+    FileMetadataManager metadataManager = new FileMetadataManager(
+        fixture.getOptionManager(),
+        true, // Use legacy wildcard expansion
+        false, // Put partitions at end
+        new Path("hdfs:///w"),
+        Lists.newArrayList(filePath));
+
+    ScanLevelProjection scanProj = new ScanLevelProjection(
+        RowSetTestUtils.projectList(SchemaPath.DYNAMIC_STAR,
+            ScanTestUtils.partitionColName(8)),
+        Lists.newArrayList(metadataManager.projectionParser()));
+
+      List<ColumnProjection> cols = scanProj.columns();
+      assertEquals(4, cols.size());
+      assertEquals(UnresolvedColumn.WILDCARD, cols.get(0).nodeType());
+      assertEquals(PartitionColumn.ID, cols.get(1).nodeType());
+      assertEquals(8, ((PartitionColumn) cols.get(1)).partition());
+      assertEquals(PartitionColumn.ID, cols.get(2).nodeType());
+      assertEquals(0, ((PartitionColumn) cols.get(2)).partition());
+      assertEquals(PartitionColumn.ID, cols.get(3).nodeType());
+      assertEquals(1, ((PartitionColumn) cols.get(3)).partition());
+  }
+
+  /**
+   * Test a case like:<br>
+   * <code>SELECT *, dir1 FROM ...</code><br>
+   * The projection list includes "dir1". The wildcard will
+   * fill in "dir0".
+   */
+
+  @Test
+  public void testLegacyWildcardAndPartitionWithOverlap() {
+    Path filePath = new Path("hdfs:///w/x/y/z.csv");
+    FileMetadataManager metadataManager = new FileMetadataManager(
+        fixture.getOptionManager(),
+        true, // Use legacy wildcard expansion
+        true, // Put partitions at end
+        new Path("hdfs:///w"),
+        Lists.newArrayList(filePath));
+
+    ScanLevelProjection scanProj = new ScanLevelProjection(
+        RowSetTestUtils.projectList(SchemaPath.DYNAMIC_STAR,
+            ScanTestUtils.partitionColName(1)),
+        Lists.newArrayList(metadataManager.projectionParser()));
+
+      List<ColumnProjection> cols = scanProj.columns();
+      assertEquals(3, cols.size());
+      assertEquals(UnresolvedColumn.WILDCARD, cols.get(0).nodeType());
+      assertEquals(PartitionColumn.ID, cols.get(1).nodeType());
+      assertEquals(0, ((PartitionColumn) cols.get(1)).partition());
+      assertEquals(PartitionColumn.ID, cols.get(2).nodeType());
+      assertEquals(1, ((PartitionColumn) cols.get(2)).partition());
+  }
+
+  @Test
+  public void testPreferedWildcardExpansionWithOverlap() {
+    Path filePath = new Path("hdfs:///w/x/y/z.csv");
+    FileMetadataManager metadataManager = new FileMetadataManager(
+        fixture.getOptionManager(),
+        true, // Use legacy wildcard expansion
+        false, // Put partitions at end
+        new Path("hdfs:///w"),
+        Lists.newArrayList(filePath));
+
+    ScanLevelProjection scanProj = new ScanLevelProjection(
+        RowSetTestUtils.projectList(SchemaPath.DYNAMIC_STAR,
+            ScanTestUtils.partitionColName(1)),
+        Lists.newArrayList(metadataManager.projectionParser()));
+
+      List<ColumnProjection> cols = scanProj.columns();
+      assertEquals(3, cols.size());
+      assertEquals(UnresolvedColumn.WILDCARD, cols.get(0).nodeType());
+      assertEquals(PartitionColumn.ID, cols.get(1).nodeType());
+      assertEquals(1, ((PartitionColumn) cols.get(1)).partition());
+      assertEquals(PartitionColumn.ID, cols.get(2).nodeType());
+      assertEquals(0, ((PartitionColumn) cols.get(2)).partition());
+  }
+
   /**
    * Verify that names that look like metadata columns, but appear
    * to be maps or arrays, are not interpreted as metadata. That is,
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestFileMetadataProjection.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestFileMetadataProjection.java
index 9161932..314bc2a 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestFileMetadataProjection.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestFileMetadataProjection.java
@@ -26,6 +26,7 @@ import static org.junit.Assert.fail;
 
 import java.util.List;
 
+import org.apache.drill.categories.RowSetTests;
 import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.physical.impl.scan.file.FileMetadata;
@@ -46,9 +47,10 @@ import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.test.SubOperatorTest;
 import org.apache.hadoop.fs.Path;
 import org.junit.Test;
-
+import org.junit.experimental.categories.Category;
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
 
+@Category(RowSetTests.class)
 public class TestFileMetadataProjection extends SubOperatorTest {
 
   @Test
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestFileScanFramework.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestFileScanFramework.java
index a10e766..2fdf3e6 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestFileScanFramework.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestFileScanFramework.java
@@ -25,6 +25,7 @@ import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 
+import org.apache.drill.categories.RowSetTests;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.common.types.TypeProtos.MinorType;
@@ -32,7 +33,7 @@ import org.apache.drill.exec.physical.impl.scan.TestScanOperatorExec.AbstractSca
 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;
-import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileReaderCreator;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileReaderFactory;
 import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
 import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
 import org.apache.drill.exec.physical.rowSet.RowSetLoader;
@@ -50,6 +51,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.FileSplit;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 /**
  * Tests the file metadata extensions to the file operator framework.
@@ -57,6 +59,7 @@ import org.junit.Test;
  * verified the underlying mechanisms.
  */
 
+@Category(RowSetTests.class)
 public class TestFileScanFramework extends SubOperatorTest {
 
   private static final String MOCK_FILE_NAME = "foo.csv";
@@ -117,7 +120,7 @@ public class TestFileScanFramework extends SubOperatorTest {
     }
   }
 
-  public static class FileScanOpFixture extends BaseFileScanOpFixture implements FileReaderCreator {
+  public static class FileScanOpFixture extends BaseFileScanOpFixture implements FileReaderFactory {
 
     protected final List<MockFileReader> readers = new ArrayList<>();
     protected Iterator<MockFileReader> readerIter;
@@ -252,7 +255,7 @@ public class TestFileScanFramework extends SubOperatorTest {
           .add("a", MinorType.INT)
           .addNullable("b", MinorType.VARCHAR, 10)
           .buildSchema();
-      schemaNegotiator.setTableSchema(schema);
+      schemaNegotiator.setTableSchema(schema, true);
       tableLoader = schemaNegotiator.build();
       return true;
     }
@@ -486,7 +489,7 @@ public class TestFileScanFramework extends SubOperatorTest {
             .add("b", MinorType.INT)
             .resumeSchema()
           .buildSchema();
-      schemaNegotiator.setTableSchema(schema);
+      schemaNegotiator.setTableSchema(schema, true);
       tableLoader = schemaNegotiator.build();
       return true;
     }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestNullColumnLoader.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestNullColumnLoader.java
deleted file mode 100644
index a413052..0000000
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestNullColumnLoader.java
+++ /dev/null
@@ -1,281 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.physical.impl.scan;
-
-import static org.junit.Assert.assertSame;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.drill.common.types.TypeProtos.DataMode;
-import org.apache.drill.common.types.TypeProtos.MajorType;
-import org.apache.drill.common.types.TypeProtos.MinorType;
-import org.apache.drill.common.types.Types;
-import org.apache.drill.exec.physical.impl.scan.project.NullColumnBuilder;
-import org.apache.drill.exec.physical.impl.scan.project.NullColumnLoader;
-import org.apache.drill.exec.physical.impl.scan.project.ResolvedNullColumn;
-import org.apache.drill.exec.physical.rowSet.ResultVectorCache;
-import org.apache.drill.exec.physical.rowSet.impl.NullResultVectorCacheImpl;
-import org.apache.drill.exec.physical.rowSet.impl.ResultVectorCacheImpl;
-import org.apache.drill.exec.record.BatchSchema;
-import org.apache.drill.exec.record.VectorContainer;
-import org.apache.drill.exec.record.metadata.SchemaBuilder;
-import org.apache.drill.exec.vector.ValueVector;
-import org.apache.drill.test.SubOperatorTest;
-import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
-import org.apache.drill.test.rowSet.RowSetComparison;
-import org.junit.Test;
-
-public class TestNullColumnLoader extends SubOperatorTest {
-
-  private ResolvedNullColumn makeNullCol(String name, MajorType nullType) {
-
-    // For this test, we don't need the projection, so just
-    // set it to null.
-
-    return new ResolvedNullColumn(name, nullType, null, 0);
-  }
-
-  private ResolvedNullColumn makeNullCol(String name) {
-    return makeNullCol(name, null);
-  }
-
-  /**
-   * Test the simplest case: default null type, nothing in the vector
-   * cache. Specify no column type, the special NULL type, or a
-   * predefined type. Output types should be set accordingly.
-   */
-
-  @Test
-  public void testBasics() {
-
-    List<ResolvedNullColumn> defns = new ArrayList<>();
-    defns.add(makeNullCol("unspecified", null));
-    defns.add(makeNullCol("nullType", Types.optional(MinorType.NULL)));
-    defns.add(makeNullCol("specifiedOpt", Types.optional(MinorType.VARCHAR)));
-    defns.add(makeNullCol("specifiedReq", Types.required(MinorType.VARCHAR)));
-    defns.add(makeNullCol("specifiedArray", Types.repeated(MinorType.VARCHAR)));
-
-    ResultVectorCache cache = new NullResultVectorCacheImpl(fixture.allocator());
-    NullColumnLoader staticLoader = new NullColumnLoader(cache, defns, null, false);
-
-    // Create a batch
-
-    VectorContainer output = staticLoader.load(2);
-
-    // Verify values and types
-
-    BatchSchema expectedSchema = new SchemaBuilder()
-        .add("unspecified", NullColumnLoader.DEFAULT_NULL_TYPE)
-        .add("nullType", NullColumnLoader.DEFAULT_NULL_TYPE)
-        .addNullable("specifiedOpt", MinorType.VARCHAR)
-        .addNullable("specifiedReq", MinorType.VARCHAR)
-        .addArray("specifiedArray", MinorType.VARCHAR)
-        .build();
-    SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
-        .addRow(null, null, null, null, new String[] {})
-        .addRow(null, null, null, null, new String[] {})
-        .build();
-
-    new RowSetComparison(expected)
-        .verifyAndClearAll(fixture.wrap(output));
-    staticLoader.close();
-  }
-
-  @Test
-  public void testCustomNullType() {
-
-    List<ResolvedNullColumn> defns = new ArrayList<>();
-    defns.add(makeNullCol("unspecified", null));
-    defns.add(makeNullCol("nullType", MajorType.newBuilder()
-        .setMinorType(MinorType.NULL)
-        .setMode(DataMode.OPTIONAL)
-        .build()));
-    defns.add(makeNullCol("nullTypeReq", MajorType.newBuilder()
-        .setMinorType(MinorType.NULL)
-        .setMode(DataMode.REQUIRED)
-        .build()));
-
-    // Null type array does not make sense, so is not tested.
-
-    ResultVectorCache cache = new NullResultVectorCacheImpl(fixture.allocator());
-    MajorType nullType = MajorType.newBuilder()
-        .setMinorType(MinorType.VARCHAR)
-        .setMode(DataMode.OPTIONAL)
-        .build();
-    NullColumnLoader staticLoader = new NullColumnLoader(cache, defns, nullType, false);
-
-    // Create a batch
-
-    VectorContainer output = staticLoader.load(2);
-
-    // Verify values and types
-
-    BatchSchema expectedSchema = new SchemaBuilder()
-        .add("unspecified", nullType)
-        .add("nullType", nullType)
-        .add("nullTypeReq", nullType)
-        .build();
-    SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
-        .addRow(null, null, null)
-        .addRow(null, null, null)
-        .build();
-
-    new RowSetComparison(expected)
-        .verifyAndClearAll(fixture.wrap(output));
-    staticLoader.close();
-  }
-
-  @Test
-  public void testCachedTypesMapToNullable() {
-
-    List<ResolvedNullColumn> defns = new ArrayList<>();
-    defns.add(makeNullCol("req"));
-    defns.add(makeNullCol("opt"));
-    defns.add(makeNullCol("rep"));
-    defns.add(makeNullCol("unk"));
-
-    // Populate the cache with a column of each mode.
-
-    ResultVectorCacheImpl cache = new ResultVectorCacheImpl(fixture.allocator());
-    cache.addOrGet(SchemaBuilder.columnSchema("req", MinorType.FLOAT8, DataMode.REQUIRED));
-    ValueVector opt = cache.addOrGet(SchemaBuilder.columnSchema("opt", MinorType.FLOAT8, DataMode.OPTIONAL));
-    ValueVector rep = cache.addOrGet(SchemaBuilder.columnSchema("rep", MinorType.FLOAT8, DataMode.REPEATED));
-
-    // Use nullable Varchar for unknown null columns.
-
-    MajorType nullType = MajorType.newBuilder()
-        .setMinorType(MinorType.VARCHAR)
-        .setMode(DataMode.OPTIONAL)
-        .build();
-    NullColumnLoader staticLoader = new NullColumnLoader(cache, defns, nullType, false);
-
-    // Create a batch
-
-    VectorContainer output = staticLoader.load(2);
-
-    // Verify vectors are reused
-
-    assertSame(opt, output.getValueVector(1).getValueVector());
-    assertSame(rep, output.getValueVector(2).getValueVector());
-
-    // Verify values and types
-
-    BatchSchema expectedSchema = new SchemaBuilder()
-        .addNullable("req", MinorType.FLOAT8)
-        .addNullable("opt", MinorType.FLOAT8)
-        .addArray("rep", MinorType.FLOAT8)
-        .addNullable("unk", MinorType.VARCHAR)
-        .build();
-    SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
-        .addRow(null, null, new int[] { }, null)
-        .addRow(null, null, new int[] { }, null)
-        .build();
-
-    new RowSetComparison(expected)
-        .verifyAndClearAll(fixture.wrap(output));
-    staticLoader.close();
-  }
-
-  @Test
-  public void testCachedTypesAllowRequired() {
-
-    List<ResolvedNullColumn> defns = new ArrayList<>();
-    defns.add(makeNullCol("req"));
-    defns.add(makeNullCol("opt"));
-    defns.add(makeNullCol("rep"));
-    defns.add(makeNullCol("unk"));
-
-    // Populate the cache with a column of each mode.
-
-    ResultVectorCacheImpl cache = new ResultVectorCacheImpl(fixture.allocator());
-    cache.addOrGet(SchemaBuilder.columnSchema("req", MinorType.FLOAT8, DataMode.REQUIRED));
-    ValueVector opt = cache.addOrGet(SchemaBuilder.columnSchema("opt", MinorType.FLOAT8, DataMode.OPTIONAL));
-    ValueVector rep = cache.addOrGet(SchemaBuilder.columnSchema("rep", MinorType.FLOAT8, DataMode.REPEATED));
-
-    // Use nullable Varchar for unknown null columns.
-
-    MajorType nullType = MajorType.newBuilder()
-        .setMinorType(MinorType.VARCHAR)
-        .setMode(DataMode.OPTIONAL)
-        .build();
-    NullColumnLoader staticLoader = new NullColumnLoader(cache, defns, nullType, true);
-
-    // Create a batch
-
-    VectorContainer output = staticLoader.load(2);
-
-    // Verify vectors are reused
-
-    assertSame(opt, output.getValueVector(1).getValueVector());
-    assertSame(rep, output.getValueVector(2).getValueVector());
-
-    // Verify values and types
-
-    BatchSchema expectedSchema = new SchemaBuilder()
-        .add("req", MinorType.FLOAT8)
-        .addNullable("opt", MinorType.FLOAT8)
-        .addArray("rep", MinorType.FLOAT8)
-        .addNullable("unk", MinorType.VARCHAR)
-        .build();
-    SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
-        .addRow(0.0, null, new int[] { }, null)
-        .addRow(0.0, null, new int[] { }, null)
-        .build();
-
-    new RowSetComparison(expected)
-        .verifyAndClearAll(fixture.wrap(output));
-    staticLoader.close();
-  }
-
-  @Test
-  public void testNullColumnBuilder() {
-
-    ResultVectorCache cache = new NullResultVectorCacheImpl(fixture.allocator());
-    NullColumnBuilder builder = new NullColumnBuilder(null, false);
-
-    builder.add("unspecified");
-    builder.add("nullType", Types.optional(MinorType.NULL));
-    builder.add("specifiedOpt", Types.optional(MinorType.VARCHAR));
-    builder.add("specifiedReq", Types.required(MinorType.VARCHAR));
-    builder.add("specifiedArray", Types.repeated(MinorType.VARCHAR));
-    builder.build(cache);
-
-    // Create a batch
-
-    builder.load(2);
-
-    // Verify values and types
-
-    BatchSchema expectedSchema = new SchemaBuilder()
-        .add("unspecified", NullColumnLoader.DEFAULT_NULL_TYPE)
-        .add("nullType", NullColumnLoader.DEFAULT_NULL_TYPE)
-        .addNullable("specifiedOpt", MinorType.VARCHAR)
-        .addNullable("specifiedReq", MinorType.VARCHAR)
-        .addArray("specifiedArray", MinorType.VARCHAR)
-        .build();
-    SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
-        .addRow(null, null, null, null, new String[] {})
-        .addRow(null, null, null, null, new String[] {})
-        .build();
-
-    new RowSetComparison(expected)
-        .verifyAndClearAll(fixture.wrap(builder.output()));
-    builder.close();
-  }
-}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestRowBatchMerger.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestRowBatchMerger.java
deleted file mode 100644
index 0794557..0000000
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestRowBatchMerger.java
+++ /dev/null
@@ -1,459 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.physical.impl.scan;
-
-import org.apache.drill.common.types.TypeProtos.MinorType;
-import org.apache.drill.common.types.Types;
-import org.apache.drill.exec.physical.impl.scan.project.NullColumnBuilder;
-import org.apache.drill.exec.physical.impl.scan.project.ResolvedMapColumn;
-import org.apache.drill.exec.physical.impl.scan.project.ResolvedColumn;
-import org.apache.drill.exec.physical.impl.scan.project.ResolvedTuple;
-import org.apache.drill.exec.physical.impl.scan.project.ResolvedTuple.ResolvedRow;
-import org.apache.drill.exec.physical.impl.scan.project.VectorSource;
-import org.apache.drill.exec.physical.rowSet.ResultVectorCache;
-import org.apache.drill.exec.physical.rowSet.impl.NullResultVectorCacheImpl;
-import org.apache.drill.exec.record.BatchSchema;
-import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.record.VectorContainer;
-import org.apache.drill.exec.record.metadata.SchemaBuilder;
-import org.apache.drill.exec.vector.ValueVector;
-import org.apache.drill.test.SubOperatorTest;
-import org.apache.drill.test.rowSet.RowSet;
-import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
-import org.apache.drill.test.rowSet.RowSetComparison;
-import org.junit.Test;
-
-import static org.apache.drill.test.rowSet.RowSetUtilities.mapValue;
-import static org.apache.drill.test.rowSet.RowSetUtilities.singleMap;
-import static org.apache.drill.test.rowSet.RowSetUtilities.mapArray;
-
-
-/**
- * Test the row batch merger by merging two batches. Tests both the
- * "direct" and "exchange" cases. Direct means that the output container
- * contains the source vector directly: they are the same vectors.
- * Exchange means we have two vectors, but we swap the underlying
- * Drillbufs to effectively shift data from source to destination
- * vector.
- */
-
-public class TestRowBatchMerger extends SubOperatorTest {
-
-  public static class RowSetSource implements VectorSource {
-
-    private SingleRowSet rowSet;
-
-    public RowSetSource(SingleRowSet rowSet) {
-      this.rowSet = rowSet;
-    }
-
-    public RowSet rowSet() { return rowSet; }
-
-    public void clear() {
-      rowSet.clear();
-    }
-
-    @Override
-    public ValueVector vector(int index) {
-      return rowSet.container().getValueVector(index).getValueVector();
-    }
-  }
-
-  private RowSetSource makeFirst() {
-    BatchSchema firstSchema = new SchemaBuilder()
-        .add("d", MinorType.VARCHAR)
-        .add("a", MinorType.INT)
-        .build();
-    return new RowSetSource(
-        fixture.rowSetBuilder(firstSchema)
-          .addRow("barney", 10)
-          .addRow("wilma", 20)
-          .build());
-  }
-
-  private RowSetSource makeSecond() {
-    BatchSchema secondSchema = new SchemaBuilder()
-        .add("b", MinorType.INT)
-        .add("c", MinorType.VARCHAR)
-        .build();
-    return new RowSetSource(
-        fixture.rowSetBuilder(secondSchema)
-          .addRow(1, "foo.csv")
-          .addRow(2, "foo.csv")
-          .build());
-  }
-
-  public static class TestProjection extends ResolvedColumn {
-
-    public TestProjection(VectorSource source, int sourceIndex) {
-      super(source, sourceIndex);
-    }
-
-    @Override
-    public String name() { return null; }
-
-    @Override
-    public int nodeType() { return -1; }
-
-    @Override
-    public MaterializedField schema() { return null; }
-  }
-
-  @Test
-  public void testSimpleFlat() {
-
-    // Create the first batch
-
-    RowSetSource first = makeFirst();
-
-    // Create the second batch
-
-    RowSetSource second = makeSecond();
-
-    ResolvedRow resolvedTuple = new ResolvedRow(null);
-    resolvedTuple.add(new TestProjection(first, 1));
-    resolvedTuple.add(new TestProjection(second, 0));
-    resolvedTuple.add(new TestProjection(second, 1));
-    resolvedTuple.add(new TestProjection(first, 0));
-
-    // Do the merge
-
-    VectorContainer output = new VectorContainer(fixture.allocator());
-    resolvedTuple.project(null, output);
-    output.setRecordCount(first.rowSet().rowCount());
-    RowSet result = fixture.wrap(output);
-
-    // Verify
-
-    BatchSchema expectedSchema = new SchemaBuilder()
-        .add("a", MinorType.INT)
-        .add("b", MinorType.INT)
-        .add("c", MinorType.VARCHAR)
-        .add("d", MinorType.VARCHAR)
-        .build();
-    SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
-        .addRow(10, 1, "foo.csv", "barney")
-        .addRow(20, 2, "foo.csv", "wilma")
-        .build();
-
-    new RowSetComparison(expected)
-      .verifyAndClearAll(result);
-  }
-
-  @Test
-  public void testImplicitFlat() {
-
-    // Create the first batch
-
-    RowSetSource first = makeFirst();
-
-    // Create the second batch
-
-    RowSetSource second = makeSecond();
-
-    ResolvedRow resolvedTuple = new ResolvedRow(null);
-    resolvedTuple.add(new TestProjection(resolvedTuple, 1));
-    resolvedTuple.add(new TestProjection(second, 0));
-    resolvedTuple.add(new TestProjection(second, 1));
-    resolvedTuple.add(new TestProjection(resolvedTuple, 0));
-
-    // Do the merge
-
-    VectorContainer output = new VectorContainer(fixture.allocator());
-    resolvedTuple.project(first.rowSet().container(), output);
-    output.setRecordCount(first.rowSet().rowCount());
-    RowSet result = fixture.wrap(output);
-
-    // Verify
-
-    BatchSchema expectedSchema = new SchemaBuilder()
-        .add("a", MinorType.INT)
-        .add("b", MinorType.INT)
-        .add("c", MinorType.VARCHAR)
-        .add("d", MinorType.VARCHAR)
-        .build();
-    SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
-        .addRow(10, 1, "foo.csv", "barney")
-        .addRow(20, 2, "foo.csv", "wilma")
-        .build();
-
-    new RowSetComparison(expected)
-      .verifyAndClearAll(result);
-  }
-
-  @Test
-  public void testFlatWithNulls() {
-
-    // Create the first batch
-
-    RowSetSource first = makeFirst();
-
-    // Create null columns
-
-    NullColumnBuilder builder = new NullColumnBuilder(null, false);
-
-    ResolvedRow resolvedTuple = new ResolvedRow(builder);
-    resolvedTuple.add(new TestProjection(resolvedTuple, 1));
-    resolvedTuple.add(resolvedTuple.nullBuilder().add("null1"));
-    resolvedTuple.add(resolvedTuple.nullBuilder().add("null2", Types.optional(MinorType.VARCHAR)));
-    resolvedTuple.add(new TestProjection(resolvedTuple, 0));
-
-    // Build the null values
-
-    ResultVectorCache cache = new NullResultVectorCacheImpl(fixture.allocator());
-    builder.build(cache);
-    builder.load(first.rowSet().rowCount());
-
-    // Do the merge
-
-    VectorContainer output = new VectorContainer(fixture.allocator());
-    resolvedTuple.project(first.rowSet().container(), output);
-    output.setRecordCount(first.rowSet().rowCount());
-    RowSet result = fixture.wrap(output);
-
-    // Verify
-
-    BatchSchema expectedSchema = new SchemaBuilder()
-        .add("a", MinorType.INT)
-        .addNullable("null1", MinorType.INT)
-        .addNullable("null2", MinorType.VARCHAR)
-        .add("d", MinorType.VARCHAR)
-        .build();
-    SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
-        .addRow(10, null, null, "barney")
-        .addRow(20, null, null, "wilma")
-        .build();
-
-    new RowSetComparison(expected)
-      .verifyAndClearAll(result);
-    builder.close();
-  }
-
-  /**
-   * Test the ability to create maps from whole cloth if requested in
-   * the projection list, and the map is not available from the data
-   * source.
-   */
-
-  @Test
-  public void testNullMaps() {
-
-    // Create the first batch
-
-    RowSetSource first = makeFirst();
-
-    // Create null columns
-
-    NullColumnBuilder builder = new NullColumnBuilder(null, false);
-    ResolvedRow resolvedTuple = new ResolvedRow(builder);
-
-    resolvedTuple.add(new TestProjection(resolvedTuple, 1));
-
-    ResolvedMapColumn nullMapCol = new ResolvedMapColumn(resolvedTuple, "map1");
-    ResolvedTuple nullMap = nullMapCol.members();
-    nullMap.add(nullMap.nullBuilder().add("null1"));
-    nullMap.add(nullMap.nullBuilder().add("null2", Types.optional(MinorType.VARCHAR)));
-
-    ResolvedMapColumn nullMapCol2 = new ResolvedMapColumn(nullMap, "map2");
-    ResolvedTuple nullMap2 = nullMapCol2.members();
-    nullMap2.add(nullMap2.nullBuilder().add("null3"));
-    nullMap.add(nullMapCol2);
-
-    resolvedTuple.add(nullMapCol);
-    resolvedTuple.add(new TestProjection(resolvedTuple, 0));
-
-    // Build the null values
-
-    ResultVectorCache cache = new NullResultVectorCacheImpl(fixture.allocator());
-    resolvedTuple.buildNulls(cache);
-
-    // LoadNulls
-
-    resolvedTuple.loadNulls(first.rowSet().rowCount());
-
-    // Do the merge
-
-    VectorContainer output = new VectorContainer(fixture.allocator());
-    resolvedTuple.project(first.rowSet().container(), output);
-    resolvedTuple.setRowCount(first.rowSet().rowCount());
-    RowSet result = fixture.wrap(output);
-
-    // Verify
-
-    BatchSchema expectedSchema = new SchemaBuilder()
-        .add("a", MinorType.INT)
-        .addMap("map1")
-          .addNullable("null1", MinorType.INT)
-          .addNullable("null2", MinorType.VARCHAR)
-          .addMap("map2")
-            .addNullable("null3", MinorType.INT)
-            .resumeMap()
-          .resumeSchema()
-        .add("d", MinorType.VARCHAR)
-        .build();
-    SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
-        .addRow(10, mapValue(null, null, singleMap(null)), "barney")
-        .addRow(20, mapValue(null, null, singleMap(null)), "wilma")
-        .build();
-
-    new RowSetComparison(expected)
-      .verifyAndClearAll(result);
-    resolvedTuple.close();
-  }
-
-  /**
-   * Test that the merger mechanism can rewrite a map to include
-   * projected null columns.
-   */
-
-  @Test
-  public void testMapRevision() {
-
-    // Create the first batch
-
-    BatchSchema inputSchema = new SchemaBuilder()
-        .add("b", MinorType.VARCHAR)
-        .addMap("a")
-          .add("c", MinorType.INT)
-          .resumeSchema()
-        .build();
-    RowSetSource input = new RowSetSource(
-        fixture.rowSetBuilder(inputSchema)
-          .addRow("barney", singleMap(10))
-          .addRow("wilma", singleMap(20))
-          .build());
-
-    // Create mappings
-
-    NullColumnBuilder builder = new NullColumnBuilder(null, false);
-    ResolvedRow resolvedTuple = new ResolvedRow(builder);
-
-    resolvedTuple.add(new TestProjection(resolvedTuple, 0));
-    ResolvedMapColumn mapCol = new ResolvedMapColumn(resolvedTuple,
-        inputSchema.getColumn(1), 1);
-    resolvedTuple.add(mapCol);
-    ResolvedTuple map = mapCol.members();
-    map.add(new TestProjection(map, 0));
-    map.add(map.nullBuilder().add("null1"));
-
-    // Build the null values
-
-    ResultVectorCache cache = new NullResultVectorCacheImpl(fixture.allocator());
-    resolvedTuple.buildNulls(cache);
-
-    // LoadNulls
-
-    resolvedTuple.loadNulls(input.rowSet().rowCount());
-
-    // Do the merge
-
-    VectorContainer output = new VectorContainer(fixture.allocator());
-    resolvedTuple.project(input.rowSet().container(), output);
-    output.setRecordCount(input.rowSet().rowCount());
-    RowSet result = fixture.wrap(output);
-
-    // Verify
-
-    BatchSchema expectedSchema = new SchemaBuilder()
-        .add("b", MinorType.VARCHAR)
-        .addMap("a")
-          .add("c", MinorType.INT)
-          .addNullable("null1", MinorType.INT)
-          .resumeSchema()
-        .build();
-    RowSet expected = fixture.rowSetBuilder(expectedSchema)
-        .addRow("barney", mapValue(10, null))
-        .addRow("wilma", mapValue(20, null))
-        .build();
-
-    new RowSetComparison(expected)
-      .verifyAndClearAll(result);
-  }
-
-  /**
-   * Test that the merger mechanism can rewrite a map array to include
-   * projected null columns.
-   */
-
-  @Test
-  public void testMapArrayRevision() {
-
-    // Create the first batch
-
-    BatchSchema inputSchema = new SchemaBuilder()
-        .add("b", MinorType.VARCHAR)
-        .addMapArray("a")
-          .add("c", MinorType.INT)
-          .resumeSchema()
-        .build();
-    RowSetSource input = new RowSetSource(
-        fixture.rowSetBuilder(inputSchema)
-          .addRow("barney", mapArray(singleMap(10), singleMap(11), singleMap(12)))
-          .addRow("wilma", mapArray(singleMap(20), singleMap(21)))
-          .build());
-
-    // Create mappings
-
-    NullColumnBuilder builder = new NullColumnBuilder(null, false);
-    ResolvedRow resolvedTuple = new ResolvedRow(builder);
-
-    resolvedTuple.add(new TestProjection(resolvedTuple, 0));
-    ResolvedMapColumn mapCol = new ResolvedMapColumn(resolvedTuple,
-        inputSchema.getColumn(1), 1);
-    resolvedTuple.add(mapCol);
-    ResolvedTuple map = mapCol.members();
-    map.add(new TestProjection(map, 0));
-    map.add(map.nullBuilder().add("null1"));
-
-    // Build the null values
-
-    ResultVectorCache cache = new NullResultVectorCacheImpl(fixture.allocator());
-    resolvedTuple.buildNulls(cache);
-
-    // LoadNulls
-
-    resolvedTuple.loadNulls(input.rowSet().rowCount());
-
-    // Do the merge
-
-    VectorContainer output = new VectorContainer(fixture.allocator());
-    resolvedTuple.project(input.rowSet().container(), output);
-    output.setRecordCount(input.rowSet().rowCount());
-    RowSet result = fixture.wrap(output);
-
-    // Verify
-
-    BatchSchema expectedSchema = new SchemaBuilder()
-        .add("b", MinorType.VARCHAR)
-        .addMapArray("a")
-          .add("c", MinorType.INT)
-          .addNullable("null1", MinorType.INT)
-          .resumeSchema()
-        .build();
-    RowSet expected = fixture.rowSetBuilder(expectedSchema)
-        .addRow("barney", mapArray(
-            mapValue(10, null), mapValue(11, null), mapValue(12, null)))
-        .addRow("wilma", mapArray(
-            mapValue(20, null), mapValue(21, null)))
-        .build();
-
-    new RowSetComparison(expected)
-      .verifyAndClearAll(result);
-  }
-
-}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanBatchWriters.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanBatchWriters.java
index 912a717..03587cb 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanBatchWriters.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanBatchWriters.java
@@ -17,6 +17,7 @@
  */
 package org.apache.drill.exec.physical.impl.scan;
 
+import org.apache.drill.categories.RowSetTests;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.ops.OperatorContext;
 import org.apache.drill.exec.physical.base.AbstractSubScan;
@@ -34,6 +35,7 @@ import org.apache.drill.test.SubOperatorTest;
 import org.apache.drill.test.rowSet.RowSet;
 import org.apache.drill.test.rowSet.RowSetComparison;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 import io.netty.buffer.DrillBuf;
 
@@ -42,6 +44,7 @@ import io.netty.buffer.DrillBuf;
  * set follows the same semantics as the original set.
  */
 
+@Category(RowSetTests.class)
 public class TestScanBatchWriters extends SubOperatorTest {
 
   @Test
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanLevelProjection.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanLevelProjection.java
deleted file mode 100644
index 558f761..0000000
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanLevelProjection.java
+++ /dev/null
@@ -1,223 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.physical.impl.scan;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import org.apache.drill.common.exceptions.UserException;
-import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.exec.physical.impl.scan.project.ScanLevelProjection;
-import org.apache.drill.exec.physical.impl.scan.project.UnresolvedColumn;
-import org.apache.drill.exec.physical.rowSet.impl.RowSetTestUtils;
-import org.apache.drill.exec.physical.rowSet.project.RequestedTuple.RequestedColumn;
-import org.apache.drill.exec.record.metadata.ProjectionType;
-import org.apache.drill.test.SubOperatorTest;
-import org.junit.Test;
-
-/**
- * Test the level of projection done at the level of the scan as a whole;
- * before knowledge of table "implicit" columns or the specific table schema.
- */
-
-public class TestScanLevelProjection extends SubOperatorTest {
-
-  /**
-   * Basic test: select a set of columns (a, b, c) when the
-   * data source has an early schema of (a, c, d). (a, c) are
-   * projected, (d) is null.
-   */
-
-  @Test
-  public void testBasics() {
-
-    // Simulate SELECT a, b, c ...
-    // Build the projection plan and verify
-
-    ScanLevelProjection scanProj = new ScanLevelProjection(
-        RowSetTestUtils.projectList("a", "b", "c"),
-        ScanTestUtils.parsers());
-    assertFalse(scanProj.projectAll());
-    assertFalse(scanProj.projectNone());
-
-    assertEquals(3, scanProj.requestedCols().size());
-    assertEquals("a", scanProj.requestedCols().get(0).rootName());
-    assertEquals("b", scanProj.requestedCols().get(1).rootName());
-    assertEquals("c", scanProj.requestedCols().get(2).rootName());
-
-    assertEquals(3, scanProj.columns().size());
-    assertEquals("a", scanProj.columns().get(0).name());
-    assertEquals("b", scanProj.columns().get(1).name());
-    assertEquals("c", scanProj.columns().get(2).name());
-
-    // Verify column type
-
-    assertEquals(UnresolvedColumn.UNRESOLVED, scanProj.columns().get(0).nodeType());
-  }
-
-  @Test
-  public void testMap() {
-    ScanLevelProjection scanProj = new ScanLevelProjection(
-        RowSetTestUtils.projectList("a.x", "b.x", "a.y", "b.y", "c"),
-        ScanTestUtils.parsers());
-    assertFalse(scanProj.projectAll());
-    assertFalse(scanProj.projectNone());
-
-    assertEquals(3, scanProj.columns().size());
-    assertEquals("a", scanProj.columns().get(0).name());
-    assertEquals("b", scanProj.columns().get(1).name());
-    assertEquals("c", scanProj.columns().get(2).name());
-
-    // Verify column type
-
-    assertEquals(UnresolvedColumn.UNRESOLVED, scanProj.columns().get(0).nodeType());
-
-    // Map structure
-
-    RequestedColumn a = ((UnresolvedColumn) scanProj.columns().get(0)).element();
-    assertTrue(a.isTuple());
-    assertEquals(ProjectionType.UNSPECIFIED, a.mapProjection().projectionType("x"));
-    assertEquals(ProjectionType.UNSPECIFIED, a.mapProjection().projectionType("y"));
-    assertEquals(ProjectionType.UNPROJECTED,  a.mapProjection().projectionType("z"));
-
-    RequestedColumn c = ((UnresolvedColumn) scanProj.columns().get(2)).element();
-    assertTrue(c.isSimple());
-  }
-
-  @Test
-  public void testArray() {
-    ScanLevelProjection scanProj = new ScanLevelProjection(
-        RowSetTestUtils.projectList("a[1]", "a[3]"),
-        ScanTestUtils.parsers());
-    assertFalse(scanProj.projectAll());
-    assertFalse(scanProj.projectNone());
-
-    assertEquals(1, scanProj.columns().size());
-    assertEquals("a", scanProj.columns().get(0).name());
-
-    // Verify column type
-
-    assertEquals(UnresolvedColumn.UNRESOLVED, scanProj.columns().get(0).nodeType());
-
-    // Map structure
-
-    RequestedColumn a = ((UnresolvedColumn) scanProj.columns().get(0)).element();
-    assertTrue(a.isArray());
-    assertFalse(a.hasIndex(0));
-    assertTrue(a.hasIndex(1));
-    assertFalse(a.hasIndex(2));
-    assertTrue(a.hasIndex(3));
-  }
-
-  /**
-   * Simulate a SELECT * query by passing "**" (Drill's internal version
-   * of the wildcard) as a column name.
-   */
-
-  @Test
-  public void testWildcard() {
-    ScanLevelProjection scanProj = new ScanLevelProjection(
-        RowSetTestUtils.projectAll(),
-        ScanTestUtils.parsers());
-
-    assertTrue(scanProj.projectAll());
-    assertFalse(scanProj.projectNone());
-    assertEquals(1, scanProj.requestedCols().size());
-    assertTrue(scanProj.requestedCols().get(0).isDynamicStar());
-
-    assertEquals(1, scanProj.columns().size());
-    assertEquals(SchemaPath.DYNAMIC_STAR, scanProj.columns().get(0).name());
-
-    // Verify bindings
-
-    assertEquals(scanProj.columns().get(0).name(), scanProj.requestedCols().get(0).rootName());
-
-    // Verify column type
-
-    assertEquals(UnresolvedColumn.WILDCARD, scanProj.columns().get(0).nodeType());
-  }
-
-  /**
-   * Test an empty projection which occurs in a
-   * SELECT COUNT(*) query.
-   */
-
-  @Test
-  public void testEmptyProjection() {
-    ScanLevelProjection scanProj = new ScanLevelProjection(
-        RowSetTestUtils.projectList(),
-        ScanTestUtils.parsers());
-
-    assertFalse(scanProj.projectAll());
-    assertTrue(scanProj.projectNone());
-    assertEquals(0, scanProj.requestedCols().size());
-  }
-
-  /**
-   * Can't include both a wildcard and a column name.
-   */
-
-  @Test
-  public void testErrorWildcardAndColumns() {
-    try {
-      new ScanLevelProjection(
-          RowSetTestUtils.projectList(SchemaPath.DYNAMIC_STAR, "a"),
-          ScanTestUtils.parsers());
-      fail();
-    } catch (IllegalArgumentException e) {
-      // Expected
-    }
-  }
-
-  /**
-   * Can't include both a column name and a wildcard.
-   */
-
-  @Test
-  public void testErrorColumnAndWildcard() {
-    try {
-      new ScanLevelProjection(
-          RowSetTestUtils.projectList("a", SchemaPath.DYNAMIC_STAR),
-          ScanTestUtils.parsers());
-      fail();
-    } catch (IllegalArgumentException e) {
-      // Expected
-    }
-  }
-
-  /**
-   * Can't include a wildcard twice.
-   * <p>
-   * Note: Drill actually allows this, but the work should be done
-   * in the project operator; scan should see at most one wildcard.
-   */
-
-  @Test
-  public void testErrorTwoWildcards() {
-    try {
-      new ScanLevelProjection(
-          RowSetTestUtils.projectList(SchemaPath.DYNAMIC_STAR, SchemaPath.DYNAMIC_STAR),
-          ScanTestUtils.parsers());
-      fail();
-    } catch (UserException e) {
-      // Expected
-    }
-  }
-}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOperatorExec.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOperatorExec.java
index 33752a7..386849b 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOperatorExec.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOperatorExec.java
@@ -27,6 +27,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
+import org.apache.drill.categories.RowSetTests;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.expression.SchemaPath;
@@ -54,6 +55,7 @@ import org.apache.drill.test.SubOperatorTest;
 import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
 import org.apache.drill.test.rowSet.RowSetComparison;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 /**
  * Test of the scan operator framework. Here the focus is on the
@@ -66,6 +68,7 @@ import org.junit.Test;
  * appear elsewhere.
  */
 
+@Category(RowSetTests.class)
 public class TestScanOperatorExec extends SubOperatorTest {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestScanOperatorExec.class);
 
@@ -187,7 +190,7 @@ public class TestScanOperatorExec extends SubOperatorTest {
           .add("a", MinorType.INT)
           .addNullable("b", MinorType.VARCHAR, 10)
           .buildSchema();
-      schemaNegotiator.setTableSchema(schema);
+      schemaNegotiator.setTableSchema(schema, true);
       tableLoader = schemaNegotiator.build();
       return true;
     }
@@ -213,7 +216,7 @@ public class TestScanOperatorExec extends SubOperatorTest {
           .add("a", MinorType.VARCHAR)
           .addNullable("b", MinorType.VARCHAR, 10)
           .buildSchema();
-      schemaNegotiator.setTableSchema(schema);
+      schemaNegotiator.setTableSchema(schema, true);
       schemaNegotiator.build();
       tableLoader = schemaNegotiator.build();
       return true;
@@ -1367,7 +1370,7 @@ public class TestScanOperatorExec extends SubOperatorTest {
       TupleMetadata schema = new SchemaBuilder()
           .add("a", MinorType.VARCHAR)
           .buildSchema();
-      schemaNegotiator.setTableSchema(schema);
+      schemaNegotiator.setTableSchema(schema, true);
       tableLoader = schemaNegotiator.build();
       return true;
     }
@@ -1493,7 +1496,7 @@ public class TestScanOperatorExec extends SubOperatorTest {
       TupleMetadata schema = new SchemaBuilder()
           .add("a", MinorType.INT)
           .buildSchema();
-      schemaNegotiator.setTableSchema(schema);
+      schemaNegotiator.setTableSchema(schema, true);
       tableLoader = schemaNegotiator.build();
       return true;
     }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOrchestratorEarlySchema.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOrchestratorEarlySchema.java
index d8c9a65..ef79b0e 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOrchestratorEarlySchema.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOrchestratorEarlySchema.java
@@ -22,12 +22,13 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
+import org.apache.drill.categories.RowSetTests;
 import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.physical.impl.protocol.SchemaTracker;
+import org.apache.drill.exec.physical.impl.scan.project.ReaderSchemaOrchestrator;
 import org.apache.drill.exec.physical.impl.scan.project.ScanSchemaOrchestrator;
-import org.apache.drill.exec.physical.impl.scan.project.ScanSchemaOrchestrator.ReaderSchemaOrchestrator;
 import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
 import org.apache.drill.exec.physical.rowSet.impl.RowSetTestUtils;
 import org.apache.drill.exec.record.BatchSchema;
@@ -37,8 +38,9 @@ import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.test.SubOperatorTest;
 import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
-import org.apache.drill.test.rowSet.RowSetComparison;
+import org.apache.drill.test.rowSet.RowSetUtilities;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 /**
  * Test the early-schema support of the scan orchestrator. "Early schema"
@@ -49,6 +51,7 @@ import org.junit.Test;
  * that tests for lower-level components have already passed.
  */
 
+@Category(RowSetTests.class)
 public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
 
   /**
@@ -78,16 +81,17 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
 
     ResultSetLoader loader = reader.makeTableLoader(tableSchema);
 
-    // Schema provided, so an empty batch is available to
-    // send downstream.
+    // Simulate a first reader in a scan that can provide an
+    // empty batch to define schema.
 
     {
+      reader.defineSchema();
       SingleRowSet expected = fixture.rowSetBuilder(tableSchema)
           .build();
 
       assertNotNull(scanner.output());
-      new RowSetComparison(expected)
-          .verifyAndClearAll(fixture.wrap(scanner.output()));
+      RowSetUtilities.verify(expected,
+          fixture.wrap(scanner.output()));
     }
 
     // Create a batch of data.
@@ -107,8 +111,8 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
           .addRow(2, "wilma")
           .build();
 
-      new RowSetComparison(expected)
-          .verifyAndClearAll(fixture.wrap(scanner.output()));
+      RowSetUtilities.verify(expected,
+          fixture.wrap(scanner.output()));
     }
 
     // Second batch.
@@ -128,8 +132,8 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
           .addRow(4, "betty")
           .build();
 
-      new RowSetComparison(expected)
-          .verifyAndClearAll(fixture.wrap(scanner.output()));
+      RowSetUtilities.verify(expected,
+          fixture.wrap(scanner.output()));
     }
 
     // Explicit reader close. (All other tests are lazy, they
@@ -167,16 +171,8 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
 
     ResultSetLoader loader = reader.makeTableLoader(tableSchema);
 
-    // Verify empty batch.
-
-    {
-      SingleRowSet expected = fixture.rowSetBuilder(tableSchema)
-          .build();
-
-      assertNotNull(scanner.output());
-      new RowSetComparison(expected)
-          .verifyAndClearAll(fixture.wrap(scanner.output()));
-    }
+    // Don't bother with an empty batch here or in other tests.
+    // Simulates the second reader in a scan.
 
     // Create a batch of data.
 
@@ -188,15 +184,13 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
 
     // Verify
 
-    {
-      SingleRowSet expected = fixture.rowSetBuilder(tableSchema)
-          .addRow(1, "fred")
-          .addRow(2, "wilma")
-          .build();
+    SingleRowSet expected = fixture.rowSetBuilder(tableSchema)
+        .addRow(1, "fred")
+        .addRow(2, "wilma")
+        .build();
 
-      new RowSetComparison(expected)
-          .verifyAndClearAll(fixture.wrap(scanner.output()));
-    }
+    RowSetUtilities.verify(expected,
+        fixture.wrap(scanner.output()));
 
     scanner.close();
   }
@@ -228,20 +222,10 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
 
     ResultSetLoader loader = reader.makeTableLoader(tableSchema);
 
-    // Verify empty batch.
-
     BatchSchema expectedSchema = new SchemaBuilder()
         .add("b", MinorType.VARCHAR)
         .add("a", MinorType.INT)
         .build();
-   {
-      SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
-          .build();
-
-      assertNotNull(scanner.output());
-      new RowSetComparison(expected)
-          .verifyAndClearAll(fixture.wrap(scanner.output()));
-   }
 
     // Create a batch of data.
 
@@ -253,17 +237,15 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
 
     // Verify
 
-   {
-     SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
-        .addRow("fred", 1)
-        .addRow("wilma", 2)
-        .build();
+   SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
+      .addRow("fred", 1)
+      .addRow("wilma", 2)
+      .build();
 
-      new RowSetComparison(expected)
-          .verifyAndClearAll(fixture.wrap(scanner.output()));
-   }
+   RowSetUtilities.verify(expected,
+       fixture.wrap(scanner.output()));
 
-    scanner.close();
+   scanner.close();
   }
 
   /**
@@ -294,21 +276,11 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
 
     ResultSetLoader loader = reader.makeTableLoader(tableSchema);
 
-    // Verify empty batch
-
     BatchSchema expectedSchema = new SchemaBuilder()
         .add("a", MinorType.INT)
         .add("b", MinorType.VARCHAR)
         .addNullable("c", MinorType.INT)
         .build();
-    {
-      SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
-          .build();
-
-      assertNotNull(scanner.output());
-      new RowSetComparison(expected)
-          .verifyAndClearAll(fixture.wrap(scanner.output()));
-   }
 
    // Create a batch of data.
 
@@ -320,15 +292,13 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
 
     // Verify
 
-    {
-      SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
-        .addRow(1, "fred", null)
-        .addRow(2, "wilma", null)
-        .build();
+    SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
+      .addRow(1, "fred", null)
+      .addRow(2, "wilma", null)
+      .build();
 
-      new RowSetComparison(expected)
-          .verifyAndClearAll(fixture.wrap(scanner.output()));
-    }
+    RowSetUtilities.verify(expected,
+        fixture.wrap(scanner.output()));
 
     scanner.close();
   }
@@ -369,21 +339,11 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
 
     ResultSetLoader loader = reader.makeTableLoader(tableSchema);
 
-    // Verify empty batch
-
     BatchSchema expectedSchema = new SchemaBuilder()
         .add("a", MinorType.INT)
         .add("b", MinorType.VARCHAR)
         .addNullable("c", MinorType.VARCHAR)
         .build();
-    {
-      SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
-          .build();
-
-      assertNotNull(scanner.output());
-      new RowSetComparison(expected)
-          .verifyAndClearAll(fixture.wrap(scanner.output()));
-   }
 
     // Create a batch of data.
 
@@ -395,15 +355,13 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
 
     // Verify
 
-    {
-      SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
-        .addRow(1, "fred", null)
-        .addRow(2, "wilma", null)
-        .build();
+    SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
+      .addRow(1, "fred", null)
+      .addRow(2, "wilma", null)
+      .build();
 
-      new RowSetComparison(expected)
-          .verifyAndClearAll(fixture.wrap(scanner.output()));
-    }
+    RowSetUtilities.verify(expected,
+        fixture.wrap(scanner.output()));
 
     scanner.close();
   }
@@ -440,19 +398,9 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
 
     assertFalse(loader.writer().column("b").schema().isProjected());
 
-    // Verify empty batch.
-
     BatchSchema expectedSchema = new SchemaBuilder()
         .add("a", MinorType.INT)
         .build();
-    {
-      SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
-          .build();
-
-      assertNotNull(scanner.output());
-      new RowSetComparison(expected)
-          .verifyAndClearAll(fixture.wrap(scanner.output()));
-    }
 
     // Create a batch of data.
 
@@ -464,15 +412,13 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
 
     // Verify
 
-    {
-      SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
-        .addRow(1)
-        .addRow(2)
-        .build();
+    SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
+      .addRow(1)
+      .addRow(2)
+      .build();
 
-      new RowSetComparison(expected)
-          .verifyAndClearAll(fixture.wrap(scanner.output()));
-    }
+    RowSetUtilities.verify(expected,
+        fixture.wrap(scanner.output()));
 
     scanner.close();
   }
@@ -516,16 +462,6 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
 
     BatchSchema expectedSchema = new SchemaBuilder()
         .build();
-    {
-      // Expect an empty schema
-
-      SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
-          .build();
-
-      assertNotNull(scanner.output());
-      new RowSetComparison(expected)
-          .verifyAndClearAll(fixture.wrap(scanner.output()));
-    }
 
     // Create a batch of data.
 
@@ -545,8 +481,8 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
         .addRow()
         .build();
 
-      new RowSetComparison(expected)
-          .verifyAndClearAll(fixture.wrap(scanner.output()));
+      RowSetUtilities.verify(expected,
+          fixture.wrap(scanner.output()));
     }
 
     // Fast path to fill in empty rows
@@ -592,18 +528,6 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
 
     reader.makeTableLoader(tableSchema);
 
-    // Schema provided, so an empty batch is available to
-    // send downstream.
-
-    {
-      SingleRowSet expected = fixture.rowSetBuilder(tableSchema)
-          .build();
-
-      assertNotNull(scanner.output());
-      new RowSetComparison(expected)
-          .verifyAndClearAll(fixture.wrap(scanner.output()));
-    }
-
     // Create a batch of data. Because there are no columns, it does
     // not make sense to ready any rows.
 
@@ -616,8 +540,8 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
       SingleRowSet expected = fixture.rowSetBuilder(tableSchema)
           .build();
 
-      new RowSetComparison(expected)
-          .verifyAndClearAll(fixture.wrap(scanner.output()));
+      RowSetUtilities.verify(expected,
+          fixture.wrap(scanner.output()));
     }
 
     scanner.close();
@@ -650,19 +574,9 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
 
     reader.makeTableLoader(tableSchema);
 
-    // Verify initial empty batch.
-
     BatchSchema expectedSchema = new SchemaBuilder()
         .addNullable("a", MinorType.INT)
         .build();
-    {
-      SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
-          .build();
-
-      assertNotNull(scanner.output());
-      new RowSetComparison(expected)
-          .verifyAndClearAll(fixture.wrap(scanner.output()));
-    }
 
     // Create a batch of data. Because there are no columns, it does
     // not make sense to ready any rows.
@@ -672,16 +586,14 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
 
     // Verify
 
-    {
-      SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
-          .build();
-
-      new RowSetComparison(expected)
-          .verifyAndClearAll(fixture.wrap(scanner.output()));
-    }
+    SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .build();
+    RowSetUtilities.verify(expected,
+        fixture.wrap(scanner.output()));
 
     scanner.close();
   }
+
   /**
    * The projection mechanism provides "type smoothing": null
    * columns prefer the type of previously-seen non-null columns.
@@ -718,6 +630,7 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
 
       ReaderSchemaOrchestrator reader = scanner.startReader();
       reader.makeTableLoader(table1Schema);
+      reader.defineSchema();
       VectorContainer output = scanner.output();
       tracker.trackSchema(output);
       schemaVersion = tracker.schemaVersion();
@@ -737,6 +650,7 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
           .buildSchema();
       ReaderSchemaOrchestrator reader = scanner.startReader();
       reader.makeTableLoader(table2Schema);
+      reader.defineSchema();
       VectorContainer output = scanner.output();
       tracker.trackSchema(output);
       assertEquals(schemaVersion, tracker.schemaVersion());
@@ -756,6 +670,7 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
           .buildSchema();
       ReaderSchemaOrchestrator reader = scanner.startReader();
       reader.makeTableLoader(table3Schema);
+      reader.defineSchema();
       VectorContainer output = scanner.output();
       tracker.trackSchema(output);
       assertEquals(schemaVersion, tracker.schemaVersion());
@@ -776,6 +691,7 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
           .buildSchema();
       ReaderSchemaOrchestrator reader = scanner.startReader();
       reader.makeTableLoader(table2Schema);
+      reader.defineSchema();
       VectorContainer output = scanner.output();
       tracker.trackSchema(output);
       assertEquals(MinorType.BIGINT, output.getSchema().getColumn(0).getType().getMinorType());
@@ -843,8 +759,8 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
           .addRow(10, "fred")
           .addRow(20, "wilma")
           .build();
-      new RowSetComparison(expected)
-        .verifyAndClearAll(fixture.wrap(projector.output()));
+      RowSetUtilities.verify(expected,
+          fixture.wrap(projector.output()));
     }
     {
       // ... FROM table 2
@@ -874,8 +790,8 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
           .addRow(30, null)
           .addRow(40, null)
           .build();
-      new RowSetComparison(expected)
-        .verifyAndClearAll(fixture.wrap(projector.output()));
+      RowSetUtilities.verify(expected,
+          fixture.wrap(projector.output()));
     }
     {
       // ... FROM table 3
@@ -899,8 +815,8 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
           .addRow(50, "dino")
           .addRow(60, "barney")
           .build();
-      new RowSetComparison(expected)
-        .verifyAndClearAll(fixture.wrap(projector.output()));
+      RowSetUtilities.verify(expected,
+          fixture.wrap(projector.output()));
     }
 
     projector.close();
@@ -926,9 +842,6 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
       ReaderSchemaOrchestrator reader = scanner.startReader();
       ResultSetLoader loader = reader.makeTableLoader(schema1);
 
-      tracker.trackSchema(scanner.output());
-      schemaVersion = tracker.schemaVersion();
-
       // Create a batch
 
       reader.startBatch();
@@ -936,6 +849,8 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
         .addRow("fred")
         .addRow("wilma");
       reader.endBatch();
+      tracker.trackSchema(scanner.output());
+      schemaVersion = tracker.schemaVersion();
 
       // Verify
 
@@ -944,8 +859,8 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
         .addRow("wilma")
         .build();
 
-      new RowSetComparison(expected)
-          .verifyAndClearAll(fixture.wrap(scanner.output()));
+      RowSetUtilities.verify(expected,
+          fixture.wrap(scanner.output()));
       scanner.closeReader();
     }
     {
@@ -960,8 +875,6 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
       ReaderSchemaOrchestrator reader = scanner.startReader();
       ResultSetLoader loader = reader.makeTableLoader(schema2);
 
-      tracker.trackSchema(scanner.output());
-      assertEquals(schemaVersion, tracker.schemaVersion());
 
       // Create a batch
 
@@ -973,13 +886,15 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
 
       // Verify, using persistent schema
 
+      tracker.trackSchema(scanner.output());
+      assertEquals(schemaVersion, tracker.schemaVersion());
       SingleRowSet expected = fixture.rowSetBuilder(schema1)
         .addRow("barney")
         .addRow("betty")
         .build();
 
-      new RowSetComparison(expected)
-          .verifyAndClearAll(fixture.wrap(scanner.output()));
+      RowSetUtilities.verify(expected,
+          fixture.wrap(scanner.output()));
       scanner.closeReader();
     }
     {
@@ -994,9 +909,6 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
       ReaderSchemaOrchestrator reader = scanner.startReader();
       ResultSetLoader loader = reader.makeTableLoader(schema3);
 
-      tracker.trackSchema(scanner.output());
-      assertEquals(schemaVersion, tracker.schemaVersion());
-
       // Create a batch
 
       reader.startBatch();
@@ -1007,13 +919,16 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
 
       // Verify, using persistent schema
 
+      tracker.trackSchema(scanner.output());
+      assertEquals(schemaVersion, tracker.schemaVersion());
+
       SingleRowSet expected = fixture.rowSetBuilder(schema1)
         .addRow("bam-bam")
         .addRow("pebbles")
         .build();
 
-      new RowSetComparison(expected)
-          .verifyAndClearAll(fixture.wrap(scanner.output()));
+      RowSetUtilities.verify(expected,
+          fixture.wrap(scanner.output()));
       scanner.closeReader();
     }
 
@@ -1073,8 +988,8 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
           .addRow(10, "fred", 110L)
           .addRow(20, "wilma", 110L)
           .build();
-      new RowSetComparison(expected)
-        .verifyAndClearAll(fixture.wrap(scanner.output()));
+      RowSetUtilities.verify(expected,
+          fixture.wrap(scanner.output()));
 
       scanner.closeReader();
     }
@@ -1100,8 +1015,8 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
           .addRow(30, "bambam", 330L)
           .addRow(40, "betty", 440L)
           .build();
-      new RowSetComparison(expected)
-        .verifyAndClearAll(fixture.wrap(scanner.output()));
+      RowSetUtilities.verify(expected,
+          fixture.wrap(scanner.output()));
     }
     {
       // ... FROM table 3
@@ -1125,8 +1040,8 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
           .addRow(50, "dino", 550L)
           .addRow(60, "barney", 660L)
           .build();
-      new RowSetComparison(expected)
-        .verifyAndClearAll(fixture.wrap(scanner.output()));
+      RowSetUtilities.verify(expected,
+          fixture.wrap(scanner.output()));
     }
 
     scanner.close();
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOrchestratorLateSchema.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOrchestratorLateSchema.java
index 0cf2cba..84ffc4e 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOrchestratorLateSchema.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOrchestratorLateSchema.java
@@ -18,10 +18,12 @@
 package org.apache.drill.exec.physical.impl.scan;
 
 import static org.junit.Assert.assertFalse;
+
+import org.apache.drill.categories.RowSetTests;
 import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.project.ReaderSchemaOrchestrator;
 import org.apache.drill.exec.physical.impl.scan.project.ScanSchemaOrchestrator;
-import org.apache.drill.exec.physical.impl.scan.project.ScanSchemaOrchestrator.ReaderSchemaOrchestrator;
 import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
 import org.apache.drill.exec.physical.rowSet.RowSetLoader;
 import org.apache.drill.exec.physical.rowSet.impl.RowSetTestUtils;
@@ -32,6 +34,7 @@ import org.apache.drill.test.SubOperatorTest;
 import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
 import org.apache.drill.test.rowSet.RowSetComparison;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 /**
  * Test the late-schema support in the scan orchestrator. "Late schema" is the case
@@ -43,6 +46,7 @@ import org.junit.Test;
  * that tests for lower-level components have already passed.
  */
 
+@Category(RowSetTests.class)
 public class TestScanOrchestratorLateSchema extends SubOperatorTest {
 
   /**
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOrchestratorMetadata.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOrchestratorMetadata.java
index e100551..c7b52e2 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOrchestratorMetadata.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOrchestratorMetadata.java
@@ -20,14 +20,15 @@ package org.apache.drill.exec.physical.impl.scan;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 
+import org.apache.drill.categories.RowSetTests;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.physical.impl.protocol.SchemaTracker;
 import org.apache.drill.exec.physical.impl.scan.file.FileMetadataManager;
+import org.apache.drill.exec.physical.impl.scan.project.ReaderSchemaOrchestrator;
 import org.apache.drill.exec.physical.impl.scan.project.ScanSchemaOrchestrator;
-import org.apache.drill.exec.physical.impl.scan.project.ScanSchemaOrchestrator.ReaderSchemaOrchestrator;
 import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
 import org.apache.drill.exec.physical.rowSet.impl.RowSetTestUtils;
 import org.apache.drill.exec.record.BatchSchema;
@@ -35,10 +36,10 @@ import org.apache.drill.exec.record.metadata.SchemaBuilder;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.test.SubOperatorTest;
 import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
-import org.apache.drill.test.rowSet.RowSetComparison;
+import org.apache.drill.test.rowSet.RowSetUtilities;
 import org.apache.hadoop.fs.Path;
 import org.junit.Test;
-
+import org.junit.experimental.categories.Category;
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
 
 /**
@@ -46,6 +47,7 @@ import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
  * with implicit file columns provided by the file metadata manager.
  */
 
+@Category(RowSetTests.class)
 public class TestScanOrchestratorMetadata extends SubOperatorTest {
 
   /**
@@ -104,8 +106,8 @@ public class TestScanOrchestratorMetadata extends SubOperatorTest {
         .addRow(2, "wilma", "/w/x/y/z.csv", "/w/x/y", "z.csv", "csv", "x", "y")
         .build();
 
-    new RowSetComparison(expected)
-        .verifyAndClearAll(fixture.wrap(scanner.output()));
+    RowSetUtilities.verify(expected,
+        fixture.wrap(scanner.output()));
 
     scanner.close();
   }
@@ -146,19 +148,9 @@ public class TestScanOrchestratorMetadata extends SubOperatorTest {
 
     ResultSetLoader loader = reader.makeTableLoader(tableSchema);
 
-    // Verify empty batch.
-
     BatchSchema expectedSchema = new SchemaBuilder()
         .addNullable("c", MinorType.INT)
         .build();
-    {
-      SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
-         .build();
-
-      assertNotNull(scanner.output());
-      new RowSetComparison(expected)
-         .verifyAndClearAll(fixture.wrap(scanner.output()));
-    }
 
     // Create a batch of data.
 
@@ -170,15 +162,13 @@ public class TestScanOrchestratorMetadata extends SubOperatorTest {
 
     // Verify
 
-    {
-      SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
-        .addSingleCol(null)
-        .addSingleCol(null)
-        .build();
+    SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
+      .addSingleCol(null)
+      .addSingleCol(null)
+      .build();
 
-      new RowSetComparison(expected)
-          .verifyAndClearAll(fixture.wrap(scanner.output()));
-    }
+    RowSetUtilities.verify(expected,
+        fixture.wrap(scanner.output()));
 
     scanner.close();
   }
@@ -229,6 +219,7 @@ public class TestScanOrchestratorMetadata extends SubOperatorTest {
 
     // Verify empty batch.
 
+    reader.defineSchema();
     BatchSchema expectedSchema = new SchemaBuilder()
         .add("a", MinorType.INT)
         .add("b", MinorType.VARCHAR)
@@ -240,8 +231,8 @@ public class TestScanOrchestratorMetadata extends SubOperatorTest {
          .build();
 
       assertNotNull(scanner.output());
-      new RowSetComparison(expected)
-         .verifyAndClearAll(fixture.wrap(scanner.output()));
+      RowSetUtilities.verify(expected,
+          fixture.wrap(scanner.output()));
     }
 
     // Create a batch of data.
@@ -260,8 +251,8 @@ public class TestScanOrchestratorMetadata extends SubOperatorTest {
         .addRow(2, "wilma", "x", "csv")
         .build();
 
-      new RowSetComparison(expected)
-          .verifyAndClearAll(fixture.wrap(scanner.output()));
+      RowSetUtilities.verify(expected,
+          fixture.wrap(scanner.output()));
     }
 
     scanner.close();
@@ -302,22 +293,12 @@ public class TestScanOrchestratorMetadata extends SubOperatorTest {
 
     ResultSetLoader loader = reader.makeTableLoader(tableSchema);
 
-    // Verify empty batch.
-
     BatchSchema expectedSchema = new SchemaBuilder()
         .addNullable("dir0", MinorType.VARCHAR)
         .add("b", MinorType.VARCHAR)
         .add("suffix", MinorType.VARCHAR)
         .addNullable("c", MinorType.INT)
         .build();
-    {
-      SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
-         .build();
-
-      assertNotNull(scanner.output());
-      new RowSetComparison(expected)
-         .verifyAndClearAll(fixture.wrap(scanner.output()));
-    }
 
     // Create a batch of data.
 
@@ -329,15 +310,13 @@ public class TestScanOrchestratorMetadata extends SubOperatorTest {
 
     // Verify
 
-    {
-      SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
-        .addRow("x", "fred", "csv", null)
-        .addRow("x", "wilma", "csv", null)
-        .build();
+    SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
+      .addRow("x", "fred", "csv", null)
+      .addRow("x", "wilma", "csv", null)
+      .build();
 
-      new RowSetComparison(expected)
-          .verifyAndClearAll(fixture.wrap(scanner.output()));
-    }
+    RowSetUtilities.verify(expected,
+        fixture.wrap(scanner.output()));
 
     scanner.close();
   }
@@ -403,8 +382,8 @@ public class TestScanOrchestratorMetadata extends SubOperatorTest {
           .addRow("x", "y", "a.csv", "fred")
           .addRow("x", "y", "a.csv", "wilma")
           .build();
-      new RowSetComparison(expected)
-        .verifyAndClearAll(fixture.wrap(scanner.output()));
+      RowSetUtilities.verify(expected,
+          fixture.wrap(scanner.output()));
 
       // Do explicit close (as in real code) to avoid an implicit
       // close which will blow away the current file info...
@@ -431,8 +410,8 @@ public class TestScanOrchestratorMetadata extends SubOperatorTest {
           .addRow("x", null, "b.csv", "bambam")
           .addRow("x", null, "b.csv", "betty")
           .build();
-      new RowSetComparison(expected)
-        .verifyAndClearAll(fixture.wrap(scanner.output()));
+      RowSetUtilities.verify(expected,
+          fixture.wrap(scanner.output()));
 
       scanner.closeReader();
     }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestSchemaLevelProjection.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestSchemaLevelProjection.java
deleted file mode 100644
index 021d7e3..0000000
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestSchemaLevelProjection.java
+++ /dev/null
@@ -1,557 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.physical.impl.scan;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertSame;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.util.List;
-
-import org.apache.drill.common.exceptions.UserException;
-import org.apache.drill.common.types.TypeProtos.MinorType;
-import org.apache.drill.exec.physical.impl.scan.project.ResolvedMapColumn;
-import org.apache.drill.exec.physical.impl.scan.project.ResolvedTuple.ResolvedRow;
-import org.apache.drill.exec.physical.impl.scan.project.ExplicitSchemaProjection;
-import org.apache.drill.exec.physical.impl.scan.project.NullColumnBuilder;
-import org.apache.drill.exec.physical.impl.scan.project.ResolvedColumn;
-import org.apache.drill.exec.physical.impl.scan.project.ResolvedNullColumn;
-import org.apache.drill.exec.physical.impl.scan.project.ResolvedTableColumn;
-import org.apache.drill.exec.physical.impl.scan.project.ResolvedTuple;
-import org.apache.drill.exec.physical.impl.scan.project.ScanLevelProjection;
-import org.apache.drill.exec.physical.impl.scan.project.UnresolvedColumn;
-import org.apache.drill.exec.physical.impl.scan.project.VectorSource;
-import org.apache.drill.exec.physical.impl.scan.project.WildcardSchemaProjection;
-import org.apache.drill.exec.physical.rowSet.impl.RowSetTestUtils;
-import org.apache.drill.exec.record.metadata.SchemaBuilder;
-import org.apache.drill.exec.record.metadata.TupleMetadata;
-import org.apache.drill.test.SubOperatorTest;
-import org.junit.Test;
-
-public class TestSchemaLevelProjection extends SubOperatorTest {
-
-  @Test
-  public void testWildcard() {
-    ScanLevelProjection scanProj = new ScanLevelProjection(
-        RowSetTestUtils.projectAll(),
-        ScanTestUtils.parsers());
-    assertEquals(1, scanProj.columns().size());
-
-    TupleMetadata tableSchema = new SchemaBuilder()
-        .add("a", MinorType.VARCHAR)
-        .addNullable("c", MinorType.INT)
-        .addArray("d", MinorType.FLOAT8)
-        .buildSchema();
-
-    NullColumnBuilder builder = new NullColumnBuilder(null, false);
-    ResolvedRow rootTuple = new ResolvedRow(builder);
-    new WildcardSchemaProjection(
-        scanProj, tableSchema, rootTuple,
-        ScanTestUtils.resolvers());
-
-    List<ResolvedColumn> columns = rootTuple.columns();
-    assertEquals(3, columns.size());
-
-    assertEquals("a", columns.get(0).name());
-    assertEquals(0, columns.get(0).sourceIndex());
-    assertSame(rootTuple, columns.get(0).source());
-    assertEquals("c", columns.get(1).name());
-    assertEquals(1, columns.get(1).sourceIndex());
-    assertSame(rootTuple, columns.get(1).source());
-    assertEquals("d", columns.get(2).name());
-    assertEquals(2, columns.get(2).sourceIndex());
-    assertSame(rootTuple, columns.get(2).source());
-  }
-
-  /**
-   * Test SELECT list with columns defined in a order and with
-   * name case different than the early-schema table.
-   */
-
-  @Test
-  public void testFullList() {
-
-    // Simulate SELECT c, b, a ...
-
-    ScanLevelProjection scanProj = new ScanLevelProjection(
-        RowSetTestUtils.projectList("c", "b", "a"),
-        ScanTestUtils.parsers());
-    assertEquals(3, scanProj.columns().size());
-
-    // Simulate a data source, with early schema, of (a, b, c)
-
-    TupleMetadata tableSchema = new SchemaBuilder()
-        .add("A", MinorType.VARCHAR)
-        .add("B", MinorType.VARCHAR)
-        .add("C", MinorType.VARCHAR)
-        .buildSchema();
-
-    NullColumnBuilder builder = new NullColumnBuilder(null, false);
-    ResolvedRow rootTuple = new ResolvedRow(builder);
-    new ExplicitSchemaProjection(
-        scanProj, tableSchema, rootTuple,
-        ScanTestUtils.resolvers());
-
-    List<ResolvedColumn> columns = rootTuple.columns();
-    assertEquals(3, columns.size());
-
-    assertEquals("c", columns.get(0).name());
-    assertEquals(2, columns.get(0).sourceIndex());
-    assertSame(rootTuple, columns.get(0).source());
-
-    assertEquals("b", columns.get(1).name());
-    assertEquals(1, columns.get(1).sourceIndex());
-    assertSame(rootTuple, columns.get(1).source());
-
-    assertEquals("a", columns.get(2).name());
-    assertEquals(0, columns.get(2).sourceIndex());
-    assertSame(rootTuple, columns.get(2).source());
-  }
-
-  /**
-   * Test SELECT list with columns missing from the table schema.
-   */
-
-  @Test
-  public void testMissing() {
-
-    // Simulate SELECT c, v, b, w ...
-
-    ScanLevelProjection scanProj = new ScanLevelProjection(
-        RowSetTestUtils.projectList("c", "v", "b", "w"),
-        ScanTestUtils.parsers());
-    assertEquals(4, scanProj.columns().size());
-
-    // Simulate a data source, with early schema, of (a, b, c)
-
-    TupleMetadata tableSchema = new SchemaBuilder()
-        .add("A", MinorType.VARCHAR)
-        .add("B", MinorType.VARCHAR)
-        .add("C", MinorType.VARCHAR)
-        .buildSchema();
-
-    NullColumnBuilder builder = new NullColumnBuilder(null, false);
-    ResolvedRow rootTuple = new ResolvedRow(builder);
-    new ExplicitSchemaProjection(
-        scanProj, tableSchema, rootTuple,
-        ScanTestUtils.resolvers());
-
-    List<ResolvedColumn> columns = rootTuple.columns();
-    assertEquals(4, columns.size());
-    VectorSource nullBuilder = rootTuple.nullBuilder();
-
-    assertEquals("c", columns.get(0).name());
-    assertEquals(2, columns.get(0).sourceIndex());
-    assertSame(rootTuple, columns.get(0).source());
-
-    assertEquals("v", columns.get(1).name());
-    assertEquals(0, columns.get(1).sourceIndex());
-    assertSame(nullBuilder, columns.get(1).source());
-
-    assertEquals("b", columns.get(2).name());
-    assertEquals(1, columns.get(2).sourceIndex());
-    assertSame(rootTuple, columns.get(2).source());
-
-    assertEquals("w", columns.get(3).name());
-    assertEquals(1, columns.get(3).sourceIndex());
-    assertSame(nullBuilder, columns.get(3).source());
-  }
-
-  @Test
-  public void testSubset() {
-
-    // Simulate SELECT c, a ...
-
-    ScanLevelProjection scanProj = new ScanLevelProjection(
-        RowSetTestUtils.projectList("c", "a"),
-        ScanTestUtils.parsers());
-    assertEquals(2, scanProj.columns().size());
-
-    // Simulate a data source, with early schema, of (a, b, c)
-
-    TupleMetadata tableSchema = new SchemaBuilder()
-        .add("A", MinorType.VARCHAR)
-        .add("B", MinorType.VARCHAR)
-        .add("C", MinorType.VARCHAR)
-        .buildSchema();
-
-    NullColumnBuilder builder = new NullColumnBuilder(null, false);
-    ResolvedRow rootTuple = new ResolvedRow(builder);
-    new ExplicitSchemaProjection(
-        scanProj, tableSchema, rootTuple,
-        ScanTestUtils.resolvers());
-
-    List<ResolvedColumn> columns = rootTuple.columns();
-    assertEquals(2, columns.size());
-
-    assertEquals("c", columns.get(0).name());
-    assertEquals(2, columns.get(0).sourceIndex());
-    assertSame(rootTuple, columns.get(0).source());
-
-    assertEquals("a", columns.get(1).name());
-    assertEquals(0, columns.get(1).sourceIndex());
-    assertSame(rootTuple, columns.get(1).source());
-  }
-
-  @Test
-  public void testDisjoint() {
-
-    // Simulate SELECT c, a ...
-
-    ScanLevelProjection scanProj = new ScanLevelProjection(
-        RowSetTestUtils.projectList("b"),
-        ScanTestUtils.parsers());
-    assertEquals(1, scanProj.columns().size());
-
-    // Simulate a data source, with early schema, of (a)
-
-    TupleMetadata tableSchema = new SchemaBuilder()
-        .add("A", MinorType.VARCHAR)
-        .buildSchema();
-
-    NullColumnBuilder builder = new NullColumnBuilder(null, false);
-    ResolvedRow rootTuple = new ResolvedRow(builder);
-    new ExplicitSchemaProjection(
-        scanProj, tableSchema, rootTuple,
-        ScanTestUtils.resolvers());
-
-    List<ResolvedColumn> columns = rootTuple.columns();
-    assertEquals(1, columns.size());
-    VectorSource nullBuilder = rootTuple.nullBuilder();
-
-    assertEquals("b", columns.get(0).name());
-    assertEquals(0, columns.get(0).sourceIndex());
-    assertSame(nullBuilder, columns.get(0).source());
-  }
-
-  @Test
-  public void testOmittedMap() {
-
-    // Simulate SELECT a, b.c.d ...
-
-    ScanLevelProjection scanProj = new ScanLevelProjection(
-        RowSetTestUtils.projectList("a", "b.c.d"),
-        ScanTestUtils.parsers());
-    assertEquals(2, scanProj.columns().size());
-    {
-      assertEquals(UnresolvedColumn.UNRESOLVED, scanProj.columns().get(1).nodeType());
-      UnresolvedColumn bCol = (UnresolvedColumn) (scanProj.columns().get(1));
-      assertTrue(bCol.element().isTuple());
-    }
-
-    // Simulate a data source, with early schema, of (a)
-
-    TupleMetadata tableSchema = new SchemaBuilder()
-        .add("a", MinorType.VARCHAR)
-        .buildSchema();
-
-    NullColumnBuilder builder = new NullColumnBuilder(null, false);
-    ResolvedRow rootTuple = new ResolvedRow(builder);
-    new ExplicitSchemaProjection(
-        scanProj, tableSchema, rootTuple,
-        ScanTestUtils.resolvers());
-
-    List<ResolvedColumn> columns = rootTuple.columns();
-    assertEquals(2, columns.size());
-
-    // Should have resolved a to a table column, b to a missing map.
-
-    // A is projected
-
-    ResolvedColumn aCol = columns.get(0);
-    assertEquals("a", aCol.name());
-    assertEquals(ResolvedTableColumn.ID, aCol.nodeType());
-
-    // B is not projected, is implicitly a map
-
-    ResolvedColumn bCol = columns.get(1);
-    assertEquals("b", bCol.name());
-    assertEquals(ResolvedMapColumn.ID, bCol.nodeType());
-
-    ResolvedMapColumn bMap = (ResolvedMapColumn) bCol;
-    ResolvedTuple bMembers = bMap.members();
-    assertNotNull(bMembers);
-    assertEquals(1, bMembers.columns().size());
-
-    // C is a map within b
-
-    ResolvedColumn cCol = bMembers.columns().get(0);
-    assertEquals(ResolvedMapColumn.ID, cCol.nodeType());
-
-    ResolvedMapColumn cMap = (ResolvedMapColumn) cCol;
-    ResolvedTuple cMembers = cMap.members();
-    assertNotNull(cMembers);
-    assertEquals(1, cMembers.columns().size());
-
-    // D is an unknown column type (not a map)
-
-    ResolvedColumn dCol = cMembers.columns().get(0);
-    assertEquals(ResolvedNullColumn.ID, dCol.nodeType());
-  }
-
-  /**
-   * Test of a map with missing columns.
-   * table of (a{b, c}), project a.c, a.d, a.e.f
-   */
-
-  @Test
-  public void testOmittedMapMembers() {
-
-    // Simulate SELECT a.c, a.d, a.e.f ...
-
-    ScanLevelProjection scanProj = new ScanLevelProjection(
-        RowSetTestUtils.projectList("x", "a.c", "a.d", "a.e.f", "y"),
-        ScanTestUtils.parsers());
-    assertEquals(3, scanProj.columns().size());
-
-    // Simulate a data source, with early schema, of (x, y, a{b, c})
-
-    TupleMetadata tableSchema = new SchemaBuilder()
-        .add("x", MinorType.VARCHAR)
-        .add("y", MinorType.INT)
-        .addMap("a")
-          .add("b", MinorType.BIGINT)
-          .add("c", MinorType.FLOAT8)
-          .resumeSchema()
-        .buildSchema();
-
-    NullColumnBuilder builder = new NullColumnBuilder(null, false);
-    ResolvedRow rootTuple = new ResolvedRow(builder);
-    new ExplicitSchemaProjection(
-        scanProj, tableSchema, rootTuple,
-        ScanTestUtils.resolvers());
-
-    List<ResolvedColumn> columns = rootTuple.columns();
-    assertEquals(3, columns.size());
-
-    // Should have resolved a.b to a map column,
-    // a.d to a missing nested map, and a.e.f to a missing
-    // nested map member
-
-    // X is projected
-
-    ResolvedColumn xCol = columns.get(0);
-    assertEquals("x", xCol.name());
-    assertEquals(ResolvedTableColumn.ID, xCol.nodeType());
-    assertSame(rootTuple, ((ResolvedTableColumn) (xCol)).source());
-    assertEquals(0, ((ResolvedTableColumn) (xCol)).sourceIndex());
-
-    // Y is projected
-
-    ResolvedColumn yCol = columns.get(2);
-    assertEquals("y", yCol.name());
-    assertEquals(ResolvedTableColumn.ID, yCol.nodeType());
-    assertSame(rootTuple, ((ResolvedTableColumn) (yCol)).source());
-    assertEquals(1, ((ResolvedTableColumn) (yCol)).sourceIndex());
-
-    // A is projected
-
-    ResolvedColumn aCol = columns.get(1);
-    assertEquals("a", aCol.name());
-    assertEquals(ResolvedMapColumn.ID, aCol.nodeType());
-
-    ResolvedMapColumn aMap = (ResolvedMapColumn) aCol;
-    ResolvedTuple aMembers = aMap.members();
-    assertFalse(aMembers.isSimpleProjection());
-    assertNotNull(aMembers);
-    assertEquals(3, aMembers.columns().size());
-
-    // a.c is projected
-
-    ResolvedColumn acCol = aMembers.columns().get(0);
-    assertEquals("c", acCol.name());
-    assertEquals(ResolvedTableColumn.ID, acCol.nodeType());
-    assertEquals(1, ((ResolvedTableColumn) (acCol)).sourceIndex());
-
-    // a.d is not in the table, is null
-
-    ResolvedColumn adCol = aMembers.columns().get(1);
-    assertEquals("d", adCol.name());
-    assertEquals(ResolvedNullColumn.ID, adCol.nodeType());
-
-    // a.e is not in the table, is implicitly a map
-
-    ResolvedColumn aeCol = aMembers.columns().get(2);
-    assertEquals("e", aeCol.name());
-    assertEquals(ResolvedMapColumn.ID, aeCol.nodeType());
-
-    ResolvedMapColumn aeMap = (ResolvedMapColumn) aeCol;
-    ResolvedTuple aeMembers = aeMap.members();
-    assertFalse(aeMembers.isSimpleProjection());
-    assertNotNull(aeMembers);
-    assertEquals(1, aeMembers.columns().size());
-
-    // a.d.f is a null column
-
-    ResolvedColumn aefCol = aeMembers.columns().get(0);
-    assertEquals("f", aefCol.name());
-    assertEquals(ResolvedNullColumn.ID, aefCol.nodeType());
-  }
-
-  /**
-   * Simple map project. This is an internal case in which the
-   * query asks for a set of columns inside a map, and the table
-   * loader produces exactly that set. No special projection is
-   * needed, the map is projected as a whole.
-   */
-
-  @Test
-  public void testSimpleMapProject() {
-
-    // Simulate SELECT a.b, a.c ...
-
-    ScanLevelProjection scanProj = new ScanLevelProjection(
-        RowSetTestUtils.projectList("a.b", "a.c"),
-        ScanTestUtils.parsers());
-    assertEquals(1, scanProj.columns().size());
-
-    // Simulate a data source, with early schema, of (a{b, c})
-
-    TupleMetadata tableSchema = new SchemaBuilder()
-        .addMap("a")
-          .add("b", MinorType.BIGINT)
-          .add("c", MinorType.FLOAT8)
-          .resumeSchema()
-        .buildSchema();
-
-    NullColumnBuilder builder = new NullColumnBuilder(null, false);
-    ResolvedRow rootTuple = new ResolvedRow(builder);
-    new ExplicitSchemaProjection(
-        scanProj, tableSchema, rootTuple,
-        ScanTestUtils.resolvers());
-
-    List<ResolvedColumn> columns = rootTuple.columns();
-    assertEquals(1, columns.size());
-
-    // Should have resolved a.b to a map column,
-    // a.d to a missing nested map, and a.e.f to a missing
-    // nested map member
-
-    // a is projected as a vector, not as a structured map
-
-    ResolvedColumn aCol = columns.get(0);
-    assertEquals("a", aCol.name());
-    assertEquals(ResolvedTableColumn.ID, aCol.nodeType());
-    assertSame(rootTuple, ((ResolvedTableColumn) (aCol)).source());
-    assertEquals(0, ((ResolvedTableColumn) (aCol)).sourceIndex());
-  }
-
-  /**
-   * Project of a non-map as a map
-   */
-
-  @Test
-  public void testMapMismatch() {
-
-    // Simulate SELECT a.b ...
-
-    ScanLevelProjection scanProj = new ScanLevelProjection(
-        RowSetTestUtils.projectList("a.b"),
-        ScanTestUtils.parsers());
-
-    // Simulate a data source, with early schema, of (a)
-    // where a is not a map.
-
-    TupleMetadata tableSchema = new SchemaBuilder()
-        .add("a", MinorType.VARCHAR)
-        .buildSchema();
-
-    NullColumnBuilder builder = new NullColumnBuilder(null, false);
-    ResolvedRow rootTuple = new ResolvedRow(builder);
-    try {
-      new ExplicitSchemaProjection(
-          scanProj, tableSchema, rootTuple,
-          ScanTestUtils.resolvers());
-      fail();
-    } catch (UserException e) {
-      // Expected
-    }
-  }
-
-  /**
-   * Test project of an array. At the scan level, we just verify
-   * that the requested column is, indeed, an array.
-   */
-
-  @Test
-  public void testArrayProject() {
-
-    // Simulate SELECT a[0] ...
-
-    ScanLevelProjection scanProj = new ScanLevelProjection(
-        RowSetTestUtils.projectList("a[0]"),
-        ScanTestUtils.parsers());
-
-    // Simulate a data source, with early schema, of (a)
-    // where a is not an array.
-
-    TupleMetadata tableSchema = new SchemaBuilder()
-        .addArray("a", MinorType.VARCHAR)
-        .buildSchema();
-
-    NullColumnBuilder builder = new NullColumnBuilder(null, false);
-    ResolvedRow rootTuple = new ResolvedRow(builder);
-    new ExplicitSchemaProjection(
-          scanProj, tableSchema, rootTuple,
-          ScanTestUtils.resolvers());
-
-    List<ResolvedColumn> columns = rootTuple.columns();
-    assertEquals(1, columns.size());
-
-    ResolvedColumn aCol = columns.get(0);
-    assertEquals("a", aCol.name());
-    assertEquals(ResolvedTableColumn.ID, aCol.nodeType());
-    assertSame(rootTuple, ((ResolvedTableColumn) (aCol)).source());
-    assertEquals(0, ((ResolvedTableColumn) (aCol)).sourceIndex());
-  }
-
-  /**
-   * Project of a non-array as an array
-   */
-
-  @Test
-  public void testArrayMismatch() {
-
-    // Simulate SELECT a[0] ...
-
-    ScanLevelProjection scanProj = new ScanLevelProjection(
-        RowSetTestUtils.projectList("a[0]"),
-        ScanTestUtils.parsers());
-
-    // Simulate a data source, with early schema, of (a)
-    // where a is not an array.
-
-    TupleMetadata tableSchema = new SchemaBuilder()
-        .add("a", MinorType.VARCHAR)
-        .buildSchema();
-
-    NullColumnBuilder builder = new NullColumnBuilder(null, false);
-    ResolvedRow rootTuple = new ResolvedRow(builder);
-    try {
-      new ExplicitSchemaProjection(
-          scanProj, tableSchema, rootTuple,
-          ScanTestUtils.resolvers());
-      fail();
-    } catch (UserException e) {
-      // Expected
-    }
-  }
-}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestSchemaSmoothing.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestSchemaSmoothing.java
deleted file mode 100644
index a30321b..0000000
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestSchemaSmoothing.java
+++ /dev/null
@@ -1,946 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.physical.impl.scan;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.util.List;
-
-import org.apache.drill.common.types.TypeProtos.MinorType;
-import org.apache.drill.exec.physical.impl.protocol.SchemaTracker;
-import org.apache.drill.exec.physical.impl.scan.file.FileMetadataColumn;
-import org.apache.drill.exec.physical.impl.scan.file.FileMetadataManager;
-import org.apache.drill.exec.physical.impl.scan.project.ExplicitSchemaProjection;
-import org.apache.drill.exec.physical.impl.scan.project.NullColumnBuilder;
-import org.apache.drill.exec.physical.impl.scan.project.ResolvedColumn;
-import org.apache.drill.exec.physical.impl.scan.project.ResolvedNullColumn;
-import org.apache.drill.exec.physical.impl.scan.project.ResolvedTableColumn;
-import org.apache.drill.exec.physical.impl.scan.project.ResolvedTuple.ResolvedRow;
-import org.apache.drill.exec.physical.impl.scan.project.ScanLevelProjection;
-import org.apache.drill.exec.physical.impl.scan.project.ScanSchemaOrchestrator;
-import org.apache.drill.exec.physical.impl.scan.project.ScanSchemaOrchestrator.ReaderSchemaOrchestrator;
-import org.apache.drill.exec.physical.impl.scan.project.SchemaSmoother;
-import org.apache.drill.exec.physical.impl.scan.project.SchemaSmoother.IncompatibleSchemaException;
-import org.apache.drill.exec.physical.impl.scan.project.SmoothingProjection;
-import org.apache.drill.exec.physical.impl.scan.project.WildcardSchemaProjection;
-import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
-import org.apache.drill.exec.physical.rowSet.impl.RowSetTestUtils;
-import org.apache.drill.exec.record.metadata.SchemaBuilder;
-import org.apache.drill.exec.record.metadata.TupleMetadata;
-import org.apache.drill.test.SubOperatorTest;
-import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
-import org.apache.drill.test.rowSet.RowSetComparison;
-import org.apache.hadoop.fs.Path;
-import org.junit.Test;
-
-import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
-
-/**
- * Tests schema smoothing at the schema projection level.
- * This level handles reusing prior types when filling null
- * values. But, because no actual vectors are involved, it
- * does not handle the schema chosen for a table ahead of
- * time, only the schema as it is merged with prior schema to
- * detect missing columns.
- * <p>
- * Focuses on the <tt>SmoothingProjection</tt> class itself.
- * <p>
- * Note that, at present, schema smoothing does not work for entire
- * maps. That is, if file 1 has, say <tt>{a: {b: 10, c: "foo"}}</tt>
- * and file 2 has, say, <tt>{a: null}</tt>, then schema smoothing does
- * not currently know how to recreate the map. The same is true of
- * lists and unions. Handling such cases is complex and is probably
- * better handled via a system that allows the user to specify their
- * intent by providing a schema to apply to the two files.
- */
-
-public class TestSchemaSmoothing extends SubOperatorTest {
-
-  /**
-   * Sanity test for the simple, discrete case. The purpose of
-   * discrete is just to run the basic lifecycle in a way that
-   * is compatible with the schema-persistence version.
-   */
-
-  @Test
-  public void testDiscrete() {
-
-    // Set up the file metadata manager
-
-    Path filePathA = new Path("hdfs:///w/x/y/a.csv");
-    Path filePathB = new Path("hdfs:///w/x/y/b.csv");
-    FileMetadataManager metadataManager = new FileMetadataManager(
-        fixture.getOptionManager(),
-        new Path("hdfs:///w"),
-        Lists.newArrayList(filePathA, filePathB));
-
-    // Set up the scan level projection
-
-    ScanLevelProjection scanProj = new ScanLevelProjection(
-        RowSetTestUtils.projectList(ScanTestUtils.FILE_NAME_COL, "a", "b"),
-        ScanTestUtils.parsers(metadataManager.projectionParser()));
-
-    {
-      // Define a file a.csv
-
-      metadataManager.startFile(filePathA);
-
-      // Build the output schema from the (a, b) table schema
-
-      TupleMetadata twoColSchema = new SchemaBuilder()
-          .add("a", MinorType.INT)
-          .addNullable("b", MinorType.VARCHAR, 10)
-          .buildSchema();
-      NullColumnBuilder builder = new NullColumnBuilder(null, false);
-      ResolvedRow rootTuple = new ResolvedRow(builder);
-      new ExplicitSchemaProjection(
-          scanProj, twoColSchema, rootTuple,
-          ScanTestUtils.resolvers(metadataManager));
-
-      // Verify the full output schema
-
-      TupleMetadata expectedSchema = new SchemaBuilder()
-          .add("filename", MinorType.VARCHAR)
-          .add("a", MinorType.INT)
-          .addNullable("b", MinorType.VARCHAR, 10)
-          .buildSchema();
-
-      // Verify
-
-      List<ResolvedColumn> columns = rootTuple.columns();
-      assertEquals(3, columns.size());
-      assertTrue(ScanTestUtils.schema(rootTuple).isEquivalent(expectedSchema));
-      assertEquals(ScanTestUtils.FILE_NAME_COL, columns.get(0).name());
-      assertEquals("a.csv", ((FileMetadataColumn) columns.get(0)).value());
-      assertEquals(ResolvedTableColumn.ID, columns.get(1).nodeType());
-    }
-    {
-      // Define a file b.csv
-
-      metadataManager.startFile(filePathB);
-
-      // Build the output schema from the (a) table schema
-
-      TupleMetadata oneColSchema = new SchemaBuilder()
-          .add("a", MinorType.INT)
-          .buildSchema();
-      NullColumnBuilder builder = new NullColumnBuilder(null, false);
-      ResolvedRow rootTuple = new ResolvedRow(builder);
-      new ExplicitSchemaProjection(
-          scanProj, oneColSchema, rootTuple,
-          ScanTestUtils.resolvers(metadataManager));
-
-      // Verify the full output schema
-      // Since this mode is "discrete", we don't remember the type
-      // of the missing column. (Instead, it is filled in at the
-      // vector level as part of vector persistence.) During projection, it is
-      // marked with type NULL so that the null column builder will fill in
-      // the proper type.
-
-      TupleMetadata expectedSchema = new SchemaBuilder()
-          .add("filename", MinorType.VARCHAR)
-          .add("a", MinorType.INT)
-          .addNullable("b", MinorType.NULL)
-          .buildSchema();
-
-      // Verify
-
-      List<ResolvedColumn> columns = rootTuple.columns();
-      assertEquals(3, columns.size());
-      assertTrue(ScanTestUtils.schema(rootTuple).isEquivalent(expectedSchema));
-      assertEquals(ScanTestUtils.FILE_NAME_COL, columns.get(0).name());
-      assertEquals("b.csv", ((FileMetadataColumn) columns.get(0)).value());
-      assertEquals(ResolvedTableColumn.ID, columns.get(1).nodeType());
-      assertEquals(ResolvedNullColumn.ID, columns.get(2).nodeType());
-    }
-  }
-
-  /**
-   * Low-level test of the smoothing projection, including the exceptions
-   * it throws when things are not going its way.
-   */
-
-  @Test
-  public void testSmoothingProjection() {
-    ScanLevelProjection scanProj = new ScanLevelProjection(
-        RowSetTestUtils.projectAll(),
-        ScanTestUtils.parsers());
-
-    // Table 1: (a: nullable bigint, b)
-
-    TupleMetadata schema1 = new SchemaBuilder()
-        .addNullable("a", MinorType.BIGINT)
-        .addNullable("b", MinorType.VARCHAR)
-        .add("c", MinorType.FLOAT8)
-        .buildSchema();
-    ResolvedRow priorSchema;
-    {
-      NullColumnBuilder builder = new NullColumnBuilder(null, false);
-      ResolvedRow rootTuple = new ResolvedRow(builder);
-      new WildcardSchemaProjection(
-          scanProj, schema1, rootTuple,
-          ScanTestUtils.resolvers());
-      priorSchema = rootTuple;
-    }
-
-    // Table 2: (a: nullable bigint, c), column omitted, original schema preserved
-
-    TupleMetadata schema2 = new SchemaBuilder()
-        .addNullable("a", MinorType.BIGINT)
-        .add("c", MinorType.FLOAT8)
-        .buildSchema();
-    try {
-      NullColumnBuilder builder = new NullColumnBuilder(null, false);
-      ResolvedRow rootTuple = new ResolvedRow(builder);
-      new SmoothingProjection(
-          scanProj, schema2, priorSchema, rootTuple,
-          ScanTestUtils.resolvers());
-      assertTrue(schema1.isEquivalent(ScanTestUtils.schema(rootTuple)));
-      priorSchema = rootTuple;
-    } catch (IncompatibleSchemaException e) {
-      fail();
-    }
-
-    // Table 3: (a, c, d), column added, must replan schema
-
-    TupleMetadata schema3 = new SchemaBuilder()
-        .addNullable("a", MinorType.BIGINT)
-        .addNullable("b", MinorType.VARCHAR)
-        .add("c", MinorType.FLOAT8)
-        .add("d", MinorType.INT)
-        .buildSchema();
-    try {
-      NullColumnBuilder builder = new NullColumnBuilder(null, false);
-      ResolvedRow rootTuple = new ResolvedRow(builder);
-      new SmoothingProjection(
-          scanProj, schema3, priorSchema, rootTuple,
-          ScanTestUtils.resolvers());
-      fail();
-    } catch (IncompatibleSchemaException e) {
-      // Expected
-    }
-
-    // Table 4: (a: double), change type must replan schema
-
-    TupleMetadata schema4 = new SchemaBuilder()
-        .addNullable("a", MinorType.FLOAT8)
-        .addNullable("b", MinorType.VARCHAR)
-        .add("c", MinorType.FLOAT8)
-        .buildSchema();
-    try {
-      NullColumnBuilder builder = new NullColumnBuilder(null, false);
-      ResolvedRow rootTuple = new ResolvedRow(builder);
-      new SmoothingProjection(
-          scanProj, schema4, priorSchema, rootTuple,
-          ScanTestUtils.resolvers());
-      fail();
-    } catch (IncompatibleSchemaException e) {
-      // Expected
-    }
-
-//    // Table 5: (a: not-nullable bigint): convert to nullable for consistency
-//
-//    TupleMetadata schema5 = new SchemaBuilder()
-//        .addNullable("a", MinorType.BIGINT)
-//        .add("c", MinorType.FLOAT8)
-//        .buildSchema();
-//    try {
-//      SmoothingProjection schemaProj = new SmoothingProjection(
-//          scanProj, schema5, dummySource, dummySource,
-//          new ArrayList<>(), priorSchema);
-//      assertTrue(schema1.isEquivalent(ScanTestUtils.schema(schemaProj.columns())));
-//    } catch (IncompatibleSchemaException e) {
-//      fail();
-//    }
-
-    // Table 6: Drop a non-nullable column, must replan
-
-    TupleMetadata schema6 = new SchemaBuilder()
-        .addNullable("a", MinorType.BIGINT)
-        .addNullable("b", MinorType.VARCHAR)
-        .buildSchema();
-    try {
-      NullColumnBuilder builder = new NullColumnBuilder(null, false);
-      ResolvedRow rootTuple = new ResolvedRow(builder);
-      new SmoothingProjection(
-          scanProj, schema6, priorSchema, rootTuple,
-          ScanTestUtils.resolvers());
-      fail();
-    } catch (IncompatibleSchemaException e) {
-      // Expected
-    }
-  }
-
-  /**
-   * Case in which the table schema is a superset of the prior
-   * schema. Discard prior schema. Turn off auto expansion of
-   * metadata for a simpler test.
-   */
-
-  @Test
-  public void testSmaller() {
-    ScanLevelProjection scanProj = new ScanLevelProjection(
-        RowSetTestUtils.projectAll(),
-        ScanTestUtils.parsers());
-
-    SchemaSmoother smoother = new SchemaSmoother(scanProj,
-        ScanTestUtils.resolvers());
-
-    TupleMetadata priorSchema = new SchemaBuilder()
-        .add("a", MinorType.INT)
-        .buildSchema();
-    TupleMetadata tableSchema = new SchemaBuilder()
-        .add("a", MinorType.INT)
-        .add("b", MinorType.VARCHAR)
-        .buildSchema();
-
-    {
-      NullColumnBuilder builder = new NullColumnBuilder(null, false);
-      ResolvedRow rootTuple = new ResolvedRow(builder);
-      smoother.resolve(priorSchema, rootTuple);
-      assertEquals(1, smoother.schemaVersion());
-      assertTrue(ScanTestUtils.schema(rootTuple).isEquivalent(priorSchema));
-    }
-    {
-      NullColumnBuilder builder = new NullColumnBuilder(null, false);
-      ResolvedRow rootTuple = new ResolvedRow(builder);
-      smoother.resolve(tableSchema, rootTuple);
-      assertEquals(2, smoother.schemaVersion());
-      assertTrue(ScanTestUtils.schema(rootTuple).isEquivalent(tableSchema));
-    }
-  }
-
-  /**
-   * Case in which the table schema and prior are disjoint
-   * sets. Discard the prior schema.
-   */
-
-  @Test
-  public void testDisjoint() {
-    ScanLevelProjection scanProj = new ScanLevelProjection(
-        RowSetTestUtils.projectAll(),
-        ScanTestUtils.parsers());
-
-    SchemaSmoother smoother = new SchemaSmoother(scanProj,
-        ScanTestUtils.resolvers());
-
-    TupleMetadata priorSchema = new SchemaBuilder()
-        .add("a", MinorType.INT)
-        .buildSchema();
-    TupleMetadata tableSchema = new SchemaBuilder()
-        .add("b", MinorType.VARCHAR)
-        .buildSchema();
-
-    {
-      doResolve(smoother, priorSchema);
-    }
-    {
-      ResolvedRow rootTuple = doResolve(smoother, tableSchema);
-      assertEquals(2, smoother.schemaVersion());
-      assertTrue(ScanTestUtils.schema(rootTuple).isEquivalent(tableSchema));
-    }
-  }
-
-  private ResolvedRow doResolve(SchemaSmoother smoother, TupleMetadata schema) {
-    NullColumnBuilder builder = new NullColumnBuilder(null, false);
-    ResolvedRow rootTuple = new ResolvedRow(builder);
-    smoother.resolve(schema, rootTuple);
-    return rootTuple;
-  }
-
-  /**
-   * Column names match, but types differ. Discard the prior schema.
-   */
-
-  @Test
-  public void testDifferentTypes() {
-    ScanLevelProjection scanProj = new ScanLevelProjection(
-        RowSetTestUtils.projectAll(),
-        ScanTestUtils.parsers());
-
-    SchemaSmoother smoother = new SchemaSmoother(scanProj,
-        ScanTestUtils.resolvers());
-
-    TupleMetadata priorSchema = new SchemaBuilder()
-        .add("a", MinorType.INT)
-        .add("b", MinorType.VARCHAR)
-        .buildSchema();
-    TupleMetadata tableSchema = new SchemaBuilder()
-        .add("a", MinorType.INT)
-        .addNullable("b", MinorType.VARCHAR)
-        .buildSchema();
-
-    {
-      doResolve(smoother, priorSchema);
-    }
-    {
-      ResolvedRow rootTuple = doResolve(smoother, tableSchema);
-      assertEquals(2, smoother.schemaVersion());
-      assertTrue(ScanTestUtils.schema(rootTuple).isEquivalent(tableSchema));
-    }
-  }
-
-  /**
-   * The prior and table schemas are identical. Preserve the prior
-   * schema (though, the output is no different than if we discarded
-   * the prior schema...)
-   */
-
-  @Test
-  public void testSameSchemas() {
-    ScanLevelProjection scanProj = new ScanLevelProjection(
-        RowSetTestUtils.projectAll(),
-        ScanTestUtils.parsers());
-
-    SchemaSmoother smoother = new SchemaSmoother(scanProj,
-        ScanTestUtils.resolvers());
-
-    TupleMetadata priorSchema = new SchemaBuilder()
-        .add("a", MinorType.INT)
-        .add("b", MinorType.VARCHAR)
-        .buildSchema();
-    TupleMetadata tableSchema = new SchemaBuilder()
-        .add("a", MinorType.INT)
-        .add("b", MinorType.VARCHAR)
-        .buildSchema();
-
-    {
-      doResolve(smoother, priorSchema);
-    }
-    {
-      ResolvedRow rootTuple = doResolve(smoother, tableSchema);
-      assertEquals(1, smoother.schemaVersion());
-      TupleMetadata actualSchema = ScanTestUtils.schema(rootTuple);
-      assertTrue(actualSchema.isEquivalent(tableSchema));
-      assertTrue(actualSchema.isEquivalent(priorSchema));
-    }
-  }
-
-  /**
-   * The prior and table schemas are identical, but the cases of names differ.
-   * Preserve the case of the first schema.
-   */
-
-  @Test
-  public void testDifferentCase() {
-    ScanLevelProjection scanProj = new ScanLevelProjection(
-        RowSetTestUtils.projectAll(),
-        ScanTestUtils.parsers());
-
-    SchemaSmoother smoother = new SchemaSmoother(scanProj,
-        ScanTestUtils.resolvers());
-
-    TupleMetadata priorSchema = new SchemaBuilder()
-        .add("a", MinorType.INT)
-        .add("b", MinorType.VARCHAR)
-        .buildSchema();
-    TupleMetadata tableSchema = new SchemaBuilder()
-        .add("A", MinorType.INT)
-        .add("B", MinorType.VARCHAR)
-        .buildSchema();
-
-    {
-      doResolve(smoother, priorSchema);
-    }
-    {
-      ResolvedRow rootTuple = doResolve(smoother, tableSchema);
-      assertEquals(1, smoother.schemaVersion());
-      assertTrue(ScanTestUtils.schema(rootTuple).isEquivalent(priorSchema));
-      List<ResolvedColumn> columns = rootTuple.columns();
-      assertEquals("a", columns.get(0).name());
-    }
-  }
-
-  /**
-   * Can't preserve the prior schema if it had required columns
-   * where the new schema has no columns.
-   */
-
-  @Test
-  public void testRequired() {
-    ScanLevelProjection scanProj = new ScanLevelProjection(
-        RowSetTestUtils.projectAll(),
-        ScanTestUtils.parsers());
-
-    SchemaSmoother smoother = new SchemaSmoother(scanProj,
-        ScanTestUtils.resolvers());
-
-    TupleMetadata priorSchema = new SchemaBuilder()
-        .add("a", MinorType.INT)
-        .addNullable("b", MinorType.VARCHAR)
-        .buildSchema();
-    TupleMetadata tableSchema = new SchemaBuilder()
-        .addNullable("b", MinorType.VARCHAR)
-        .buildSchema();
-
-    {
-      doResolve(smoother, priorSchema);
-    }
-    {
-      ResolvedRow rootTuple = doResolve(smoother, tableSchema);
-      assertEquals(2, smoother.schemaVersion());
-      assertTrue(ScanTestUtils.schema(rootTuple).isEquivalent(tableSchema));
-    }
-  }
-
-  /**
-   * Preserve the prior schema if table is a subset and missing columns
-   * are nullable or repeated.
-   */
-
-  @Test
-  public void testMissingNullableColumns() {
-    ScanLevelProjection scanProj = new ScanLevelProjection(
-        RowSetTestUtils.projectAll(),
-        ScanTestUtils.parsers());
-
-    SchemaSmoother smoother = new SchemaSmoother(scanProj,
-        ScanTestUtils.resolvers());
-
-    TupleMetadata priorSchema = new SchemaBuilder()
-        .addNullable("a", MinorType.INT)
-        .add("b", MinorType.VARCHAR)
-        .addArray("c", MinorType.BIGINT)
-        .buildSchema();
-    TupleMetadata tableSchema = new SchemaBuilder()
-        .add("b", MinorType.VARCHAR)
-        .buildSchema();
-
-    {
-      doResolve(smoother, priorSchema);
-    }
-    {
-      ResolvedRow rootTuple = doResolve(smoother, tableSchema);
-      assertEquals(1, smoother.schemaVersion());
-      assertTrue(ScanTestUtils.schema(rootTuple).isEquivalent(priorSchema));
-    }
-  }
-
-  /**
-   * Preserve the prior schema if table is a subset. Map the table
-   * columns to the output using the prior schema ordering.
-   */
-
-  @Test
-  public void testReordering() {
-    ScanLevelProjection scanProj = new ScanLevelProjection(
-        RowSetTestUtils.projectAll(),
-        ScanTestUtils.parsers());
-
-    SchemaSmoother smoother = new SchemaSmoother(scanProj,
-        ScanTestUtils.resolvers());
-
-    TupleMetadata priorSchema = new SchemaBuilder()
-        .addNullable("a", MinorType.INT)
-        .add("b", MinorType.VARCHAR)
-        .addArray("c", MinorType.BIGINT)
-        .buildSchema();
-    TupleMetadata tableSchema = new SchemaBuilder()
-        .add("b", MinorType.VARCHAR)
-        .addNullable("a", MinorType.INT)
-        .buildSchema();
-
-    {
-      doResolve(smoother, priorSchema);
-    }
-    {
-      ResolvedRow rootTuple = doResolve(smoother, tableSchema);
-      assertEquals(1, smoother.schemaVersion());
-      assertTrue(ScanTestUtils.schema(rootTuple).isEquivalent(priorSchema));
-    }
-  }
-
-  /**
-   * If using the legacy wildcard expansion, reuse schema if partition paths
-   * are the same length.
-   */
-
-  @Test
-  public void testSamePartitionLength() {
-
-    // Set up the file metadata manager
-
-    Path filePathA = new Path("hdfs:///w/x/y/a.csv");
-    Path filePathB = new Path("hdfs:///w/x/y/b.csv");
-    FileMetadataManager metadataManager = new FileMetadataManager(
-        fixture.getOptionManager(),
-        new Path("hdfs:///w"),
-        Lists.newArrayList(filePathA, filePathB));
-
-    // Set up the scan level projection
-
-    ScanLevelProjection scanProj = new ScanLevelProjection(
-        ScanTestUtils.projectAllWithMetadata(2),
-        ScanTestUtils.parsers(metadataManager.projectionParser()));
-
-    // Define the schema smoother
-
-    SchemaSmoother smoother = new SchemaSmoother(scanProj,
-        ScanTestUtils.resolvers(metadataManager));
-
-    TupleMetadata tableSchema = new SchemaBuilder()
-        .add("a", MinorType.INT)
-        .add("b", MinorType.VARCHAR)
-        .buildSchema();
-
-    TupleMetadata expectedSchema = ScanTestUtils.expandMetadata(tableSchema, metadataManager, 2);
-    {
-      metadataManager.startFile(filePathA);
-      ResolvedRow rootTuple = doResolve(smoother, tableSchema);
-      assertTrue(ScanTestUtils.schema(rootTuple).isEquivalent(expectedSchema));
-    }
-    {
-      metadataManager.startFile(filePathB);
-      ResolvedRow rootTuple = doResolve(smoother, tableSchema);
-      assertEquals(1, smoother.schemaVersion());
-      assertTrue(ScanTestUtils.schema(rootTuple).isEquivalent(expectedSchema));
-    }
-  }
-
-  /**
-   * If using the legacy wildcard expansion, reuse schema if the new partition path
-   * is shorter than the previous. (Unneeded partitions will be set to null by the
-   * scan projector.)
-   */
-
-  @Test
-  public void testShorterPartitionLength() {
-
-    // Set up the file metadata manager
-
-    Path filePathA = new Path("hdfs:///w/x/y/a.csv");
-    Path filePathB = new Path("hdfs:///w/x/b.csv");
-    FileMetadataManager metadataManager = new FileMetadataManager(
-        fixture.getOptionManager(),
-        new Path("hdfs:///w"),
-        Lists.newArrayList(filePathA, filePathB));
-
-    // Set up the scan level projection
-
-    ScanLevelProjection scanProj = new ScanLevelProjection(
-        ScanTestUtils.projectAllWithMetadata(2),
-        ScanTestUtils.parsers(metadataManager.projectionParser()));
-
-    // Define the schema smoother
-
-    SchemaSmoother smoother = new SchemaSmoother(scanProj,
-        ScanTestUtils.resolvers(metadataManager));
-
-    TupleMetadata tableSchema = new SchemaBuilder()
-        .add("a", MinorType.INT)
-        .add("b", MinorType.VARCHAR)
-        .buildSchema();
-
-    TupleMetadata expectedSchema = ScanTestUtils.expandMetadata(tableSchema, metadataManager, 2);
-    {
-      metadataManager.startFile(filePathA);
-      ResolvedRow rootTuple = doResolve(smoother, tableSchema);
-      assertTrue(ScanTestUtils.schema(rootTuple).isEquivalent(expectedSchema));
-    }
-    {
-      metadataManager.startFile(filePathB);
-      ResolvedRow rootTuple = doResolve(smoother, tableSchema);
-      assertEquals(1, smoother.schemaVersion());
-      assertTrue(ScanTestUtils.schema(rootTuple).isEquivalent(expectedSchema));
-    }
-  }
-
-  /**
-   * If using the legacy wildcard expansion, we are able to use the same
-   * schema even if the new partition path is longer than the previous.
-   * Because all file names are provided up front.
-   */
-
-  @Test
-  public void testLongerPartitionLength() {
-
-    // Set up the file metadata manager
-
-    Path filePathA = new Path("hdfs:///w/x/a.csv");
-    Path filePathB = new Path("hdfs:///w/x/y/b.csv");
-    FileMetadataManager metadataManager = new FileMetadataManager(
-        fixture.getOptionManager(),
-        new Path("hdfs:///w"),
-        Lists.newArrayList(filePathA, filePathB));
-
-    // Set up the scan level projection
-
-    ScanLevelProjection scanProj = new ScanLevelProjection(
-        ScanTestUtils.projectAllWithMetadata(2),
-        ScanTestUtils.parsers(metadataManager.projectionParser()));
-
-    // Define the schema smoother
-
-    SchemaSmoother smoother = new SchemaSmoother(scanProj,
-        ScanTestUtils.resolvers(metadataManager));
-
-    TupleMetadata tableSchema = new SchemaBuilder()
-        .add("a", MinorType.INT)
-        .add("b", MinorType.VARCHAR)
-        .buildSchema();
-
-    TupleMetadata expectedSchema = ScanTestUtils.expandMetadata(tableSchema, metadataManager, 2);
-    {
-      metadataManager.startFile(filePathA);
-      ResolvedRow rootTuple = doResolve(smoother, tableSchema);
-      assertTrue(ScanTestUtils.schema(rootTuple).isEquivalent(expectedSchema));
-    }
-    {
-      metadataManager.startFile(filePathB);
-      ResolvedRow rootTuple = doResolve(smoother, tableSchema);
-      assertEquals(1, smoother.schemaVersion());
-      assertTrue(ScanTestUtils.schema(rootTuple).isEquivalent(expectedSchema));
-    }
-  }
-
-  /**
-   * Integrated test across multiple schemas at the batch level.
-   */
-
-  @Test
-  public void testSmoothableSchemaBatches() {
-    ScanLevelProjection scanProj = new ScanLevelProjection(
-        RowSetTestUtils.projectAll(),
-        ScanTestUtils.parsers());
-
-    SchemaSmoother smoother = new SchemaSmoother(scanProj,
-        ScanTestUtils.resolvers());
-
-    // Table 1: (a: bigint, b)
-
-    TupleMetadata schema1 = new SchemaBuilder()
-        .addNullable("a", MinorType.BIGINT)
-        .addNullable("b", MinorType.VARCHAR)
-        .add("c", MinorType.FLOAT8)
-        .buildSchema();
-    {
-      ResolvedRow rootTuple = doResolve(smoother, schema1);
-
-      // Just use the original schema.
-
-      assertTrue(schema1.isEquivalent(ScanTestUtils.schema(rootTuple)));
-      assertEquals(1, smoother.schemaVersion());
-    }
-
-    // Table 2: (a: nullable bigint, c), column ommitted, original schema preserved
-
-    TupleMetadata schema2 = new SchemaBuilder()
-        .addNullable("a", MinorType.BIGINT)
-        .add("c", MinorType.FLOAT8)
-        .buildSchema();
-    {
-      ResolvedRow rootTuple = doResolve(smoother, schema2);
-      assertTrue(schema1.isEquivalent(ScanTestUtils.schema(rootTuple)));
-      assertEquals(1, smoother.schemaVersion());
-    }
-
-    // Table 3: (a, c, d), column added, must replan schema
-
-    TupleMetadata schema3 = new SchemaBuilder()
-        .addNullable("a", MinorType.BIGINT)
-        .addNullable("b", MinorType.VARCHAR)
-        .add("c", MinorType.FLOAT8)
-        .add("d", MinorType.INT)
-        .buildSchema();
-    {
-      ResolvedRow rootTuple = doResolve(smoother, schema3);
-      assertTrue(schema3.isEquivalent(ScanTestUtils.schema(rootTuple)));
-      assertEquals(2, smoother.schemaVersion());
-    }
-
-    // Table 4: Drop a non-nullable column, must replan
-
-    TupleMetadata schema4 = new SchemaBuilder()
-        .addNullable("a", MinorType.BIGINT)
-        .addNullable("b", MinorType.VARCHAR)
-        .buildSchema();
-    {
-      ResolvedRow rootTuple = doResolve(smoother, schema4);
-      assertTrue(schema4.isEquivalent(ScanTestUtils.schema(rootTuple)));
-      assertEquals(3, smoother.schemaVersion());
-    }
-
-    // Table 5: (a: double), change type must replan schema
-
-    TupleMetadata schema5 = new SchemaBuilder()
-        .addNullable("a", MinorType.FLOAT8)
-        .addNullable("b", MinorType.VARCHAR)
-        .buildSchema();
-    {
-      ResolvedRow rootTuple = doResolve(smoother, schema5);
-      assertTrue(schema5.isEquivalent(ScanTestUtils.schema(rootTuple)));
-       assertEquals(4, smoother.schemaVersion());
-    }
-
-//    // Table 6: (a: not-nullable bigint): convert to nullable for consistency
-//
-//    TupleMetadata schema6 = new SchemaBuilder()
-//        .add("a", MinorType.FLOAT8)
-//        .add("b", MinorType.VARCHAR)
-//        .buildSchema();
-//    {
-//      SchemaLevelProjection schemaProj = smoother.resolve(schema3, dummySource);
-//      assertTrue(schema5.isEquivalent(ScanTestUtils.schema(schemaProj.columns())));
-//    }
-  }
-
-  /**
-   * A SELECT * query uses the schema of the table as the output schema.
-   * This is trivial when the scanner has one table. But, if two or more
-   * tables occur, then things get interesting. The first table sets the
-   * schema. The second table then has:
-   * <ul>
-   * <li>The same schema, trivial case.</li>
-   * <li>A subset of the first table. The type of the "missing" column
-   * from the first table is used for a null column in the second table.</li>
-   * <li>A superset or disjoint set of the first schema. This triggers a hard schema
-   * change.</li>
-   * </ul>
-   * <p>
-   * It is an open question whether previous columns should be preserved on
-   * a hard reset. For now, the code implements, and this test verifies, that a
-   * hard reset clears the "memory" of prior schemas.
-   */
-
-  @Test
-  public void testWildcardSmoothing() {
-    ScanSchemaOrchestrator projector = new ScanSchemaOrchestrator(fixture.allocator());
-    projector.enableSchemaSmoothing(true);
-    projector.build(RowSetTestUtils.projectAll());
-
-    TupleMetadata firstSchema = new SchemaBuilder()
-        .add("a", MinorType.INT)
-        .addNullable("b", MinorType.VARCHAR, 10)
-        .addNullable("c", MinorType.BIGINT)
-        .buildSchema();
-    TupleMetadata subsetSchema = new SchemaBuilder()
-        .addNullable("b", MinorType.VARCHAR, 10)
-        .add("a", MinorType.INT)
-        .buildSchema();
-    TupleMetadata disjointSchema = new SchemaBuilder()
-        .add("a", MinorType.INT)
-        .addNullable("b", MinorType.VARCHAR, 10)
-        .add("d", MinorType.VARCHAR)
-        .buildSchema();
-
-    SchemaTracker tracker = new SchemaTracker();
-    int schemaVersion;
-    {
-      // First table, establishes the baseline
-      // ... FROM table 1
-
-      ReaderSchemaOrchestrator reader = projector.startReader();
-      ResultSetLoader loader = reader.makeTableLoader(firstSchema);
-
-      reader.startBatch();
-      loader.writer()
-          .addRow(10, "fred", 110L)
-          .addRow(20, "wilma", 110L);
-      reader.endBatch();
-
-      tracker.trackSchema(projector.output());
-      schemaVersion = tracker.schemaVersion();
-
-      SingleRowSet expected = fixture.rowSetBuilder(firstSchema)
-          .addRow(10, "fred", 110L)
-          .addRow(20, "wilma", 110L)
-          .build();
-      new RowSetComparison(expected)
-        .verifyAndClearAll(fixture.wrap(projector.output()));
-    }
-    {
-      // Second table, same schema, the trivial case
-      // ... FROM table 2
-
-      ReaderSchemaOrchestrator reader = projector.startReader();
-      ResultSetLoader loader = reader.makeTableLoader(firstSchema);
-
-      reader.startBatch();
-      loader.writer()
-          .addRow(70, "pebbles", 770L)
-          .addRow(80, "hoppy", 880L);
-      reader.endBatch();
-
-      tracker.trackSchema(projector.output());
-      assertEquals(schemaVersion, tracker.schemaVersion());
-
-      SingleRowSet expected = fixture.rowSetBuilder(firstSchema)
-          .addRow(70, "pebbles", 770L)
-          .addRow(80, "hoppy", 880L)
-          .build();
-      new RowSetComparison(expected)
-        .verifyAndClearAll(fixture.wrap(projector.output()));
-    }
-    {
-      // Third table: subset schema of first two
-      // ... FROM table 3
-
-      ReaderSchemaOrchestrator reader = projector.startReader();
-      ResultSetLoader loader = reader.makeTableLoader(subsetSchema);
-
-      reader.startBatch();
-      loader.writer()
-          .addRow("bambam", 30)
-          .addRow("betty", 40);
-      reader.endBatch();
-
-      tracker.trackSchema(projector.output());
-      assertEquals(schemaVersion, tracker.schemaVersion());
-
-      SingleRowSet expected = fixture.rowSetBuilder(firstSchema)
-          .addRow(30, "bambam", null)
-          .addRow(40, "betty", null)
-          .build();
-      new RowSetComparison(expected)
-        .verifyAndClearAll(fixture.wrap(projector.output()));
-    }
-    {
-      // Fourth table: disjoint schema, cases a schema reset
-      // ... FROM table 4
-
-      ReaderSchemaOrchestrator reader = projector.startReader();
-      ResultSetLoader loader = reader.makeTableLoader(disjointSchema);
-
-      reader.startBatch();
-      loader.writer()
-          .addRow(50, "dino", "supporting")
-          .addRow(60, "barney", "main");
-      reader.endBatch();
-
-      tracker.trackSchema(projector.output());
-      assertNotEquals(schemaVersion, tracker.schemaVersion());
-
-      SingleRowSet expected = fixture.rowSetBuilder(disjointSchema)
-          .addRow(50, "dino", "supporting")
-          .addRow(60, "barney", "main")
-          .build();
-      new RowSetComparison(expected)
-        .verifyAndClearAll(fixture.wrap(projector.output()));
-    }
-
-    projector.close();
-  }
-
-  // TODO: Test schema smoothing with repeated
-  // TODO: Test hard schema change
-  // TODO: Typed null column tests (resurrect)
-  // TODO: Test maps and arrays of maps
-}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/project/TestConstantColumnLoader.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/project/TestConstantColumnLoader.java
index b45374b..086da96 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/project/TestConstantColumnLoader.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/project/TestConstantColumnLoader.java
@@ -24,14 +24,21 @@ import org.apache.drill.categories.RowSetTests;
 import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.ScanTestUtils;
+import org.apache.drill.exec.physical.impl.scan.file.FileMetadata;
+import org.apache.drill.exec.physical.impl.scan.file.FileMetadataColumn;
+import org.apache.drill.exec.physical.impl.scan.file.FileMetadataColumnDefn;
+import org.apache.drill.exec.physical.impl.scan.file.PartitionColumn;
 import org.apache.drill.exec.physical.impl.scan.project.ConstantColumnLoader.ConstantColumnSpec;
 import org.apache.drill.exec.physical.rowSet.impl.ResultVectorCacheImpl;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.store.ColumnExplorer.ImplicitFileColumns;
 import org.apache.drill.test.SubOperatorTest;
 import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
 import org.apache.drill.test.rowSet.RowSetComparison;
+import org.apache.hadoop.fs.Path;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -114,4 +121,42 @@ public class TestConstantColumnLoader extends SubOperatorTest {
         .verifyAndClearAll(fixture.wrap(staticLoader.load(2)));
     staticLoader.close();
   }
+
+  @Test
+  public void testFileMetadata() {
+
+    FileMetadata fileInfo = new FileMetadata(new Path("hdfs:///w/x/y/z.csv"), new Path("hdfs:///w"));
+    List<ConstantColumnSpec> defns = new ArrayList<>();
+    FileMetadataColumnDefn iDefn = new FileMetadataColumnDefn(
+        ScanTestUtils.SUFFIX_COL, ImplicitFileColumns.SUFFIX);
+    FileMetadataColumn iCol = new FileMetadataColumn(ScanTestUtils.SUFFIX_COL,
+        iDefn, fileInfo, null, 0);
+    defns.add(iCol);
+
+    String partColName = ScanTestUtils.partitionColName(1);
+    PartitionColumn pCol = new PartitionColumn(partColName, 1, fileInfo, null, 0);
+    defns.add(pCol);
+
+    ResultVectorCacheImpl cache = new ResultVectorCacheImpl(fixture.allocator());
+    ConstantColumnLoader staticLoader = new ConstantColumnLoader(cache, defns);
+
+    // Create a batch
+
+    staticLoader.load(2);
+
+    // Verify
+
+    BatchSchema expectedSchema = new SchemaBuilder()
+        .add(ScanTestUtils.SUFFIX_COL, MinorType.VARCHAR)
+        .addNullable(partColName, MinorType.VARCHAR)
+        .build();
+    SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addRow("csv", "y")
+        .addRow("csv", "y")
+        .build();
+
+    new RowSetComparison(expected)
+        .verifyAndClearAll(fixture.wrap(staticLoader.load(2)));
+    staticLoader.close();
+  }
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/project/TestScanLevelProjection.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/project/TestScanLevelProjection.java
index 5b49ab3..f40e847 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/project/TestScanLevelProjection.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/project/TestScanLevelProjection.java
@@ -19,6 +19,7 @@ package org.apache.drill.exec.physical.impl.scan.project;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -27,6 +28,8 @@ import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.physical.impl.scan.ScanTestUtils;
 import org.apache.drill.exec.physical.rowSet.impl.RowSetTestUtils;
+import org.apache.drill.exec.physical.rowSet.project.ImpliedTupleRequest;
+import org.apache.drill.exec.physical.rowSet.project.RequestedTuple;
 import org.apache.drill.exec.physical.rowSet.project.RequestedTuple.RequestedColumn;
 import org.apache.drill.exec.record.metadata.ProjectionType;
 import org.apache.drill.test.SubOperatorTest;
@@ -56,6 +59,7 @@ public class TestScanLevelProjection extends SubOperatorTest {
     final ScanLevelProjection scanProj = new ScanLevelProjection(
         RowSetTestUtils.projectList("a", "b", "c"),
         ScanTestUtils.parsers());
+
     assertFalse(scanProj.projectAll());
     assertFalse(scanProj.projectNone());
 
@@ -72,6 +76,19 @@ public class TestScanLevelProjection extends SubOperatorTest {
     // Verify column type
 
     assertEquals(UnresolvedColumn.UNRESOLVED, scanProj.columns().get(0).nodeType());
+
+    // Verify tuple projection
+
+    RequestedTuple outputProj = scanProj.rootProjection();
+    assertEquals(3, outputProj.projections().size());
+    assertNotNull(outputProj.get("a"));
+    assertTrue(outputProj.get("a").isSimple());
+
+    RequestedTuple readerProj = scanProj.readerProjection();
+    assertEquals(3, readerProj.projections().size());
+    assertNotNull(readerProj.get("a"));
+    assertEquals(ProjectionType.UNSPECIFIED, readerProj.projectionType("a"));
+    assertEquals(ProjectionType.UNPROJECTED, readerProj.projectionType("d"));
   }
 
   /**
@@ -85,6 +102,7 @@ public class TestScanLevelProjection extends SubOperatorTest {
     final ScanLevelProjection scanProj = new ScanLevelProjection(
         RowSetTestUtils.projectList("a.x", "b.x", "a.y", "b.y", "c"),
         ScanTestUtils.parsers());
+
     assertFalse(scanProj.projectAll());
     assertFalse(scanProj.projectNone());
 
@@ -107,6 +125,20 @@ public class TestScanLevelProjection extends SubOperatorTest {
 
     final RequestedColumn c = ((UnresolvedColumn) scanProj.columns().get(2)).element();
     assertTrue(c.isSimple());
+
+    // Verify tuple projection
+
+    RequestedTuple outputProj = scanProj.rootProjection();
+    assertEquals(3, outputProj.projections().size());
+    assertNotNull(outputProj.get("a"));
+    assertTrue(outputProj.get("a").isTuple());
+
+    RequestedTuple readerProj = scanProj.readerProjection();
+    assertEquals(3, readerProj.projections().size());
+    assertNotNull(readerProj.get("a"));
+    assertEquals(ProjectionType.TUPLE, readerProj.projectionType("a"));
+    assertEquals(ProjectionType.UNSPECIFIED, readerProj.projectionType("c"));
+    assertEquals(ProjectionType.UNPROJECTED, readerProj.projectionType("d"));
   }
 
   /**
@@ -119,6 +151,7 @@ public class TestScanLevelProjection extends SubOperatorTest {
     final ScanLevelProjection scanProj = new ScanLevelProjection(
         RowSetTestUtils.projectList("a[1]", "a[3]"),
         ScanTestUtils.parsers());
+
     assertFalse(scanProj.projectAll());
     assertFalse(scanProj.projectNone());
 
@@ -137,6 +170,19 @@ public class TestScanLevelProjection extends SubOperatorTest {
     assertTrue(a.hasIndex(1));
     assertFalse(a.hasIndex(2));
     assertTrue(a.hasIndex(3));
+
+    // Verify tuple projection
+
+    RequestedTuple outputProj = scanProj.rootProjection();
+    assertEquals(1, outputProj.projections().size());
+    assertNotNull(outputProj.get("a"));
+    assertTrue(outputProj.get("a").isArray());
+
+    RequestedTuple readerProj = scanProj.readerProjection();
+    assertEquals(1, readerProj.projections().size());
+    assertNotNull(readerProj.get("a"));
+    assertEquals(ProjectionType.ARRAY, readerProj.projectionType("a"));
+    assertEquals(ProjectionType.UNPROJECTED, readerProj.projectionType("c"));
   }
 
   /**
@@ -165,6 +211,17 @@ public class TestScanLevelProjection extends SubOperatorTest {
     // Verify column type
 
     assertEquals(UnresolvedColumn.WILDCARD, scanProj.columns().get(0).nodeType());
+
+    // Verify tuple projection
+
+    RequestedTuple outputProj = scanProj.rootProjection();
+    assertEquals(1, outputProj.projections().size());
+    assertNotNull(outputProj.get("**"));
+    assertTrue(outputProj.get("**").isWildcard());
+
+    RequestedTuple readerProj = scanProj.readerProjection();
+    assertTrue(readerProj instanceof ImpliedTupleRequest);
+    assertEquals(ProjectionType.UNSPECIFIED, readerProj.projectionType("a"));
   }
 
   /**
@@ -181,38 +238,62 @@ public class TestScanLevelProjection extends SubOperatorTest {
     assertFalse(scanProj.projectAll());
     assertTrue(scanProj.projectNone());
     assertEquals(0, scanProj.requestedCols().size());
+
+    // Verify tuple projection
+
+    RequestedTuple outputProj = scanProj.rootProjection();
+    assertEquals(0, outputProj.projections().size());
+
+    RequestedTuple readerProj = scanProj.readerProjection();
+    assertTrue(readerProj instanceof ImpliedTupleRequest);
+    assertEquals(ProjectionType.UNPROJECTED, readerProj.projectionType("a"));
   }
 
   /**
-   * Can't include both a wildcard and a column name.
+   * Can include both a wildcard and a column name. The Project
+   * operator will fill in the column, the scan framework just ignores
+   * the extra column.
    */
 
   @Test
-  public void testErrorWildcardAndColumns() {
-    try {
-      new ScanLevelProjection(
+  public void testWildcardAndColumns() {
+    ScanLevelProjection scanProj = new ScanLevelProjection(
           RowSetTestUtils.projectList(SchemaPath.DYNAMIC_STAR, "a"),
           ScanTestUtils.parsers());
-      fail();
-    } catch (final IllegalArgumentException e) {
-      // Expected
-    }
+
+    assertTrue(scanProj.projectAll());
+    assertFalse(scanProj.projectNone());
+    assertEquals(2, scanProj.requestedCols().size());
+    assertEquals(1, scanProj.columns().size());
+
+    // Verify tuple projection
+
+    RequestedTuple outputProj = scanProj.rootProjection();
+    assertEquals(2, outputProj.projections().size());
+    assertNotNull(outputProj.get("**"));
+    assertTrue(outputProj.get("**").isWildcard());
+    assertNotNull(outputProj.get("a"));
+
+    RequestedTuple readerProj = scanProj.readerProjection();
+    assertTrue(readerProj instanceof ImpliedTupleRequest);
+    assertEquals(ProjectionType.UNSPECIFIED, readerProj.projectionType("a"));
+    assertEquals(ProjectionType.UNSPECIFIED, readerProj.projectionType("c"));
   }
 
   /**
-   * Can't include both a column name and a wildcard.
+   * Test a column name and a wildcard.
    */
 
   @Test
-  public void testErrorColumnAndWildcard() {
-    try {
-      new ScanLevelProjection(
+  public void testColumnAndWildcard() {
+    ScanLevelProjection scanProj = new ScanLevelProjection(
           RowSetTestUtils.projectList("a", SchemaPath.DYNAMIC_STAR),
           ScanTestUtils.parsers());
-      fail();
-    } catch (final IllegalArgumentException e) {
-      // Expected
-    }
+
+    assertTrue(scanProj.projectAll());
+    assertFalse(scanProj.projectNone());
+    assertEquals(2, scanProj.requestedCols().size());
+    assertEquals(1, scanProj.columns().size());
   }
 
   /**
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/project/TestSchemaSmoothing.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/project/TestSchemaSmoothing.java
index a21b1e4..8adc037 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/project/TestSchemaSmoothing.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/project/TestSchemaSmoothing.java
@@ -28,8 +28,9 @@ import org.apache.drill.categories.RowSetTests;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.physical.impl.protocol.SchemaTracker;
 import org.apache.drill.exec.physical.impl.scan.ScanTestUtils;
+import org.apache.drill.exec.physical.impl.scan.file.FileMetadataColumn;
+import org.apache.drill.exec.physical.impl.scan.file.FileMetadataManager;
 import org.apache.drill.exec.physical.impl.scan.project.ResolvedTuple.ResolvedRow;
-import org.apache.drill.exec.physical.impl.scan.project.ScanSchemaOrchestrator.ReaderSchemaOrchestrator;
 import org.apache.drill.exec.physical.impl.scan.project.SchemaSmoother.IncompatibleSchemaException;
 import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
 import org.apache.drill.exec.physical.rowSet.impl.RowSetTestUtils;
@@ -38,8 +39,10 @@ import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.test.SubOperatorTest;
 import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
 import org.apache.drill.test.rowSet.RowSetComparison;
+import org.apache.hadoop.fs.Path;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
 
 /**
  * Tests schema smoothing at the schema projection level.
@@ -63,7 +66,7 @@ import org.junit.experimental.categories.Category;
  * to a fundamental limitation in Drill:
  * <ul>
  * <li>Drill cannot predict the future: each file (or batch)
- * may have a different schema.</ul>
+ * may have a different schema.</li>
  * <li>Drill does not know about these differences until they
  * occur.</li>
  * <li>The scan operator is obligated to return the same schema
@@ -85,6 +88,105 @@ import org.junit.experimental.categories.Category;
 public class TestSchemaSmoothing extends SubOperatorTest {
 
   /**
+   * Sanity test for the simple, discrete case. The purpose of
+   * discrete is just to run the basic lifecycle in a way that
+   * is compatible with the schema-persistence version.
+   */
+
+  @Test
+  public void testDiscrete() {
+
+    // Set up the file metadata manager
+
+    Path filePathA = new Path("hdfs:///w/x/y/a.csv");
+    Path filePathB = new Path("hdfs:///w/x/y/b.csv");
+    FileMetadataManager metadataManager = new FileMetadataManager(
+        fixture.getOptionManager(),
+        new Path("hdfs:///w"),
+        Lists.newArrayList(filePathA, filePathB));
+
+    // Set up the scan level projection
+
+    ScanLevelProjection scanProj = new ScanLevelProjection(
+        RowSetTestUtils.projectList(ScanTestUtils.FILE_NAME_COL, "a", "b"),
+        ScanTestUtils.parsers(metadataManager.projectionParser()));
+
+    {
+      // Define a file a.csv
+
+      metadataManager.startFile(filePathA);
+
+      // Build the output schema from the (a, b) table schema
+
+      TupleMetadata twoColSchema = new SchemaBuilder()
+          .add("a", MinorType.INT)
+          .addNullable("b", MinorType.VARCHAR, 10)
+          .buildSchema();
+      NullColumnBuilder builder = new NullColumnBuilder(null, false);
+      ResolvedRow rootTuple = new ResolvedRow(builder);
+      new ExplicitSchemaProjection(
+          scanProj, twoColSchema, rootTuple,
+          ScanTestUtils.resolvers(metadataManager));
+
+      // Verify the full output schema
+
+      TupleMetadata expectedSchema = new SchemaBuilder()
+          .add("filename", MinorType.VARCHAR)
+          .add("a", MinorType.INT)
+          .addNullable("b", MinorType.VARCHAR, 10)
+          .buildSchema();
+
+      // Verify
+
+      List<ResolvedColumn> columns = rootTuple.columns();
+      assertEquals(3, columns.size());
+      assertTrue(ScanTestUtils.schema(rootTuple).isEquivalent(expectedSchema));
+      assertEquals(ScanTestUtils.FILE_NAME_COL, columns.get(0).name());
+      assertEquals("a.csv", ((FileMetadataColumn) columns.get(0)).value());
+      assertEquals(ResolvedTableColumn.ID, columns.get(1).nodeType());
+    }
+    {
+      // Define a file b.csv
+
+      metadataManager.startFile(filePathB);
+
+      // Build the output schema from the (a) table schema
+
+      TupleMetadata oneColSchema = new SchemaBuilder()
+          .add("a", MinorType.INT)
+          .buildSchema();
+      NullColumnBuilder builder = new NullColumnBuilder(null, false);
+      ResolvedRow rootTuple = new ResolvedRow(builder);
+      new ExplicitSchemaProjection(
+          scanProj, oneColSchema, rootTuple,
+          ScanTestUtils.resolvers(metadataManager));
+
+      // Verify the full output schema
+      // Since this mode is "discrete", we don't remember the type
+      // of the missing column. (Instead, it is filled in at the
+      // vector level as part of vector persistence.) During projection, it is
+      // marked with type NULL so that the null column builder will fill in
+      // the proper type.
+
+      TupleMetadata expectedSchema = new SchemaBuilder()
+          .add("filename", MinorType.VARCHAR)
+          .add("a", MinorType.INT)
+          .addNullable("b", MinorType.NULL)
+          .buildSchema();
+
+      // Verify
+
+      List<ResolvedColumn> columns = rootTuple.columns();
+      assertEquals(3, columns.size());
+      assertTrue(ScanTestUtils.schema(rootTuple).isEquivalent(expectedSchema));
+      assertEquals(ScanTestUtils.FILE_NAME_COL, columns.get(0).name());
+      assertEquals("b.csv", ((FileMetadataColumn) columns.get(0)).value());
+      assertEquals(ResolvedTableColumn.ID, columns.get(1).nodeType());
+      assertEquals(ResolvedNullColumn.ID, columns.get(2).nodeType());
+    }
+  }
+
+  /**
    * Low-level test of the smoothing projection, including the exceptions
    * it throws when things are not going its way.
    */
@@ -463,6 +565,150 @@ public class TestSchemaSmoothing extends SubOperatorTest {
       assertTrue(ScanTestUtils.schema(rootTuple).isEquivalent(priorSchema));
     }
   }
+
+  /**
+   * If using the legacy wildcard expansion, reuse schema if partition paths
+   * are the same length.
+   */
+
+  @Test
+  public void testSamePartitionLength() {
+
+    // Set up the file metadata manager
+
+    Path filePathA = new Path("hdfs:///w/x/y/a.csv");
+    Path filePathB = new Path("hdfs:///w/x/y/b.csv");
+    FileMetadataManager metadataManager = new FileMetadataManager(
+        fixture.getOptionManager(),
+        new Path("hdfs:///w"),
+        Lists.newArrayList(filePathA, filePathB));
+
+    // Set up the scan level projection
+
+    ScanLevelProjection scanProj = new ScanLevelProjection(
+        ScanTestUtils.projectAllWithMetadata(2),
+        ScanTestUtils.parsers(metadataManager.projectionParser()));
+
+    // Define the schema smoother
+
+    SchemaSmoother smoother = new SchemaSmoother(scanProj,
+        ScanTestUtils.resolvers(metadataManager));
+
+    TupleMetadata tableSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .add("b", MinorType.VARCHAR)
+        .buildSchema();
+
+    TupleMetadata expectedSchema = ScanTestUtils.expandMetadata(tableSchema, metadataManager, 2);
+    {
+      metadataManager.startFile(filePathA);
+      ResolvedRow rootTuple = doResolve(smoother, tableSchema);
+      assertTrue(ScanTestUtils.schema(rootTuple).isEquivalent(expectedSchema));
+    }
+    {
+      metadataManager.startFile(filePathB);
+      ResolvedRow rootTuple = doResolve(smoother, tableSchema);
+      assertEquals(1, smoother.schemaVersion());
+      assertTrue(ScanTestUtils.schema(rootTuple).isEquivalent(expectedSchema));
+    }
+  }
+
+  /**
+   * If using the legacy wildcard expansion, reuse schema if the new partition path
+   * is shorter than the previous. (Unneeded partitions will be set to null by the
+   * scan projector.)
+   */
+
+  @Test
+  public void testShorterPartitionLength() {
+
+    // Set up the file metadata manager
+
+    Path filePathA = new Path("hdfs:///w/x/y/a.csv");
+    Path filePathB = new Path("hdfs:///w/x/b.csv");
+    FileMetadataManager metadataManager = new FileMetadataManager(
+        fixture.getOptionManager(),
+        new Path("hdfs:///w"),
+        Lists.newArrayList(filePathA, filePathB));
+
+    // Set up the scan level projection
+
+    ScanLevelProjection scanProj = new ScanLevelProjection(
+        ScanTestUtils.projectAllWithMetadata(2),
+        ScanTestUtils.parsers(metadataManager.projectionParser()));
+
+    // Define the schema smoother
+
+    SchemaSmoother smoother = new SchemaSmoother(scanProj,
+        ScanTestUtils.resolvers(metadataManager));
+
+    TupleMetadata tableSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .add("b", MinorType.VARCHAR)
+        .buildSchema();
+
+    TupleMetadata expectedSchema = ScanTestUtils.expandMetadata(tableSchema, metadataManager, 2);
+    {
+      metadataManager.startFile(filePathA);
+      ResolvedRow rootTuple = doResolve(smoother, tableSchema);
+      assertTrue(ScanTestUtils.schema(rootTuple).isEquivalent(expectedSchema));
+    }
+    {
+      metadataManager.startFile(filePathB);
+      ResolvedRow rootTuple = doResolve(smoother, tableSchema);
+      assertEquals(1, smoother.schemaVersion());
+      assertTrue(ScanTestUtils.schema(rootTuple).isEquivalent(expectedSchema));
+    }
+  }
+
+  /**
+   * If using the legacy wildcard expansion, we are able to use the same
+   * schema even if the new partition path is longer than the previous.
+   * Because all file names are provided up front.
+   */
+
+  @Test
+  public void testLongerPartitionLength() {
+
+    // Set up the file metadata manager
+
+    Path filePathA = new Path("hdfs:///w/x/a.csv");
+    Path filePathB = new Path("hdfs:///w/x/y/b.csv");
+    FileMetadataManager metadataManager = new FileMetadataManager(
+        fixture.getOptionManager(),
+        new Path("hdfs:///w"),
+        Lists.newArrayList(filePathA, filePathB));
+
+    // Set up the scan level projection
+
+    ScanLevelProjection scanProj = new ScanLevelProjection(
+        ScanTestUtils.projectAllWithMetadata(2),
+        ScanTestUtils.parsers(metadataManager.projectionParser()));
+
+    // Define the schema smoother
+
+    SchemaSmoother smoother = new SchemaSmoother(scanProj,
+        ScanTestUtils.resolvers(metadataManager));
+
+    TupleMetadata tableSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .add("b", MinorType.VARCHAR)
+        .buildSchema();
+
+    TupleMetadata expectedSchema = ScanTestUtils.expandMetadata(tableSchema, metadataManager, 2);
+    {
+      metadataManager.startFile(filePathA);
+      ResolvedRow rootTuple = doResolve(smoother, tableSchema);
+      assertTrue(ScanTestUtils.schema(rootTuple).isEquivalent(expectedSchema));
+    }
+    {
+      metadataManager.startFile(filePathB);
+      ResolvedRow rootTuple = doResolve(smoother, tableSchema);
+      assertEquals(1, smoother.schemaVersion());
+      assertTrue(ScanTestUtils.schema(rootTuple).isEquivalent(expectedSchema));
+    }
+  }
+
   /**
    * Integrated test across multiple schemas at the batch level.
    */