You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by ar...@apache.org on 2020/03/11 14:38:34 UTC

[drill] 01/02: DRILL-7601: Shift column conversion to reader from scan framework

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

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

commit 88a84b9d088dad549e8489c487c55c9d7aecff9a
Author: Paul Rogers <pa...@yahoo.com>
AuthorDate: Fri Feb 21 11:00:14 2020 -0800

    DRILL-7601: Shift column conversion to reader from scan framework
    
    Allows the column writers to be generic, moves scan-specific
    conversions into each reader where needed, implemented in
    a reader-specific way.
    
    Adds a revised way of handling projections in the result set
    loader that is not coupled with conversion, as the prior
    design was.
    
    Updates the CSV, Avro, Log and HDF5 readers.
    
    closes #1993
---
 .../drill/exec/store/esri/ShpBatchReader.java      |   6 +-
 .../drill/exec/store/esri/ShpFormatPlugin.java     |   2 +-
 .../drill/exec/store/excel/ExcelFormatPlugin.java  |   2 +-
 .../drill/exec/store/hdf5/HDF5BatchReader.java     | 113 ++--
 .../drill/exec/store/hdf5/HDF5FormatPlugin.java    |   2 +-
 .../exec/store/hdf5/writers/HDF5DataWriter.java    |  35 +-
 .../store/hdf5/writers/HDF5DoubleDataWriter.java   |  34 +-
 .../store/hdf5/writers/HDF5EnumDataWriter.java     |  18 +-
 .../store/hdf5/writers/HDF5FloatDataWriter.java    |  32 +-
 .../exec/store/hdf5/writers/HDF5IntDataWriter.java |  35 +-
 .../store/hdf5/writers/HDF5LongDataWriter.java     |  34 +-
 .../exec/store/hdf5/writers/HDF5MapDataWriter.java |  49 +-
 .../store/hdf5/writers/HDF5StringDataWriter.java   |  28 +-
 .../hdf5/writers/HDF5TimestampDataWriter.java      |  18 +-
 .../drill/exec/store/hdf5/writers/WriterSpec.java  |  62 +++
 .../drill/exec/ops/ExecutorFragmentContext.java    |   4 +-
 .../drill/exec/physical/impl/ImplCreator.java      |   2 +-
 .../physical/impl/OperatorCreatorRegistry.java     |   5 +-
 .../impl/metadata/MetadataHandlerBatch.java        |  10 +-
 .../impl/scan/columns/ColumnsArrayParser.java      |   5 -
 .../scan}/convert/AbstractConvertFromString.java   | 105 ++--
 .../impl/scan/convert/ColumnConverter.java         |  46 ++
 .../impl/scan}/convert/ConvertBooleanToString.java |   4 +-
 .../impl/scan}/convert/ConvertDateToString.java    |   4 +-
 .../impl/scan}/convert/ConvertDecimalToString.java |   4 +-
 .../impl/scan}/convert/ConvertDoubleToString.java  |   4 +-
 .../impl/scan}/convert/ConvertIntToString.java     |   4 +-
 .../scan}/convert/ConvertIntervalToString.java     |   4 +-
 .../impl/scan}/convert/ConvertLongToString.java    |   4 +-
 .../impl/scan}/convert/ConvertStringToBoolean.java |   2 +-
 .../impl/scan}/convert/ConvertStringToDate.java    |   2 +-
 .../impl/scan}/convert/ConvertStringToDecimal.java |   2 +-
 .../impl/scan}/convert/ConvertStringToDouble.java  |   2 +-
 .../impl/scan}/convert/ConvertStringToInt.java     |   2 +-
 .../scan}/convert/ConvertStringToInterval.java     |   2 +-
 .../impl/scan}/convert/ConvertStringToLong.java    |   2 +-
 .../impl/scan}/convert/ConvertStringToTime.java    |   2 +-
 .../scan}/convert/ConvertStringToTimeStamp.java    |   2 +-
 .../scan}/convert/ConvertTimeStampToString.java    |   4 +-
 .../impl/scan}/convert/ConvertTimeToString.java    |   4 +-
 .../impl/scan/convert/DirectConverter.java}        |  67 +--
 .../impl/scan/convert/StandardConversions.java     | 478 ++++++++++++++++
 .../physical/impl/scan/convert/package-info.java   |  78 +++
 .../impl/scan/file/FileMetadataColumnsParser.java  |   4 +-
 .../physical/impl/scan/file/FileScanFramework.java |  33 +-
 ...dataManager.java => ImplicitColumnManager.java} |  12 +-
 .../physical/impl/scan/file/PartitionColumn.java   |   1 -
 .../impl/scan/framework/ManagedScanFramework.java  |   8 +-
 .../impl/scan/framework/SchemaNegotiator.java      | 126 +++--
 .../impl/scan/framework/SchemaNegotiatorImpl.java  |  35 +-
 .../impl/scan/framework/ShimBatchReader.java       |   7 +-
 .../scan/project/AbstractUnresolvedColumn.java     |   3 -
 .../impl/scan/project/ReaderLevelProjection.java   |   2 -
 .../scan/project/ReaderSchemaOrchestrator.java     |  50 +-
 .../physical/impl/scan/project/ResolvedColumn.java |   2 -
 .../impl/scan/project/ResolvedMapColumn.java       |   1 -
 .../impl/scan/project/ResolvedNullColumn.java      |   3 -
 .../impl/scan/project/ScanLevelProjection.java     |  57 +-
 .../impl/scan/project/ScanSchemaOrchestrator.java  |  62 +--
 .../impl/scan/project/StaticColumnLoader.java      |   6 +-
 .../project/projSet/AbstractProjectionSet.java     |  80 ---
 .../scan/project/projSet/AbstractReadColProj.java  |  46 --
 .../scan/project/projSet/EmptyProjectionSet.java   |  51 --
 .../project/projSet/ExplicitProjectionSet.java     | 118 ----
 .../scan/project/projSet/ProjectedDictColumn.java  |  39 --
 .../scan/project/projSet/ProjectedMapColumn.java   |  39 --
 .../scan/project/projSet/ProjectedReadColumn.java  |  73 ---
 .../scan/project/projSet/ProjectionSetBuilder.java | 101 ----
 .../scan/project/projSet/ProjectionSetFactory.java |  79 ---
 .../impl/scan/project/projSet/TypeConverter.java   | 175 ------
 .../project/projSet/UnprojectedReadColumn.java     |  41 --
 .../project/projSet/WildcardProjectionSet.java     |  70 ---
 .../impl/scan/project/projSet/package-info.java    |  99 ----
 .../exec/physical/resultSet/ProjectionSet.java     |  96 ----
 .../exec/physical/resultSet/ResultSetLoader.java   |  22 +-
 .../exec/physical/resultSet/ResultVectorCache.java |   2 +-
 .../physical/resultSet/impl/ColumnBuilder.java     | 264 ++++-----
 .../exec/physical/resultSet/impl/ColumnState.java  |  82 ++-
 .../physical/resultSet/impl/ContainerState.java    |  14 +-
 .../physical/resultSet/impl/LoaderInternals.java   |  17 +-
 .../resultSet/impl/NullResultVectorCacheImpl.java  |   2 +-
 .../physical/resultSet/impl/ProjectionFilter.java  | 241 ++++++++
 .../physical/resultSet/impl/RepeatedListState.java |  15 +-
 .../resultSet/impl/ResultSetCopierImpl.java        |  10 +-
 .../resultSet/impl/ResultSetLoaderImpl.java        | 320 +++++------
 ...ionBuilder.java => ResultSetOptionBuilder.java} |  34 +-
 .../resultSet/impl/ResultVectorCacheImpl.java      |   2 +-
 .../physical/resultSet/impl/RowSetLoaderImpl.java  |   5 +-
 .../exec/physical/resultSet/impl/TupleState.java   |  17 +-
 .../resultSet/model/single/BaseWriterBuilder.java  |  11 +-
 .../resultSet/project/ImpliedTupleRequest.java     |  17 +
 .../project}/ProjectionChecker.java                |   4 +-
 .../physical/resultSet/project/Projections.java    |  10 +
 .../physical/resultSet/project/RequestedTuple.java |  16 +
 .../resultSet/project/RequestedTupleImpl.java      |  26 +
 .../drill/exec/physical/rowSet/DirectRowSet.java   |  12 +-
 .../drill/exec/physical/rowSet/RowSetBuilder.java  |  24 +-
 .../exec/physical/rowSet/RowSetWriterImpl.java     |   7 +-
 .../drill/exec/store/SchemaTreeProvider.java       |   2 +-
 .../drill/exec/store/avro/AvroBatchReader.java     |  16 +-
 .../drill/exec/store/avro/AvroFormatPlugin.java    |   2 +-
 .../drill/exec/store/avro/ColumnConverter.java     | 117 +---
 .../exec/store/avro/ColumnConverterFactory.java    | 257 +++++++++
 .../exec/store/avro/ColumnConvertersUtil.java      | 115 ----
 .../exec/store/dfs/easy/EasyFormatPlugin.java      |  59 +-
 .../exec/store/easy/text/TextFormatPlugin.java     |  24 +-
 .../store/easy/text/reader/BaseFieldOutput.java    |  10 +-
 .../easy/text/reader/CompliantTextBatchReader.java | 127 ++++-
 .../easy/text/reader/ConstrainedFieldOutput.java}  |  31 +-
 .../store/easy/text/reader/FieldVarCharOutput.java |  18 +-
 .../exec/store/easy/text/reader/TextInput.java     |  13 +-
 .../easy/text/reader/TextParsingSettings.java      |  85 ++-
 .../exec/store/easy/text/reader/TextReader.java    |   6 +-
 .../drill/exec/store/log/LogBatchReader.java       |  57 +-
 .../drill/exec/store/log/LogFormatPlugin.java      | 117 ++--
 .../exec/store/mock/ExtendedMockBatchReader.java   |   4 +-
 .../exec/store/mock/MockScanBatchCreator.java      |   4 +-
 .../drill/exec/store/pcap/PcapBatchReader.java     |   6 +-
 .../drill/exec/store/pcap/PcapFormatPlugin.java    |   2 +-
 .../impl/scan/BaseScanOperatorExecTest.java        |   2 +-
 .../exec/physical/impl/scan/ScanTestUtils.java     |  14 +-
 .../exec/physical/impl/scan/TestColumnsArray.java  |  18 +-
 .../impl/scan/TestColumnsArrayFramework.java       |   6 +-
 .../physical/impl/scan/TestColumnsArrayParser.java |  12 +-
 .../physical/impl/scan/TestFileScanFramework.java  |  42 +-
 ...mnParser.java => TestImplicitColumnParser.java} |  66 +--
 ...tion.java => TestImplicitColumnProjection.java} |  35 +-
 .../impl/scan/TestScanOperExecOuputSchema.java     | 114 ++--
 .../impl/scan/TestScanOperExecOverflow.java        |   6 +-
 .../impl/scan/TestScanOperExecSmoothing.java       |   4 +-
 .../impl/scan/TestScanOrchestratorEarlySchema.java |  30 +-
 ...va => TestScanOrchestratorImplicitColumns.java} |  46 +-
 .../impl/scan/TestScanOrchestratorLateSchema.java  |   6 +-
 .../impl/scan/convert/TestColumnConverter.java     | 157 ++++++
 .../scan/convert/TestDirectConverter.java}         | 383 +++++--------
 .../impl/scan/project/TestNullColumnLoader.java    |  12 +-
 .../scan/project/TestReaderLevelProjection.java    |   2 +-
 .../impl/scan/project/TestScanLevelProjection.java | 214 ++++++--
 .../impl/scan/project/TestSchemaSmoothing.java     |  20 +-
 .../scan/project/projSet/TestProjectionSet.java    | 604 ---------------------
 .../resultSet/impl/TestResultSetCopier.java        |  10 +-
 .../impl/TestResultSetLoaderDictArray.java         |  58 +-
 .../resultSet/impl/TestResultSetLoaderDicts.java   |  70 +--
 .../impl/TestResultSetLoaderEmptyProject.java      |  42 +-
 .../resultSet/impl/TestResultSetLoaderLimits.java  |  28 +-
 .../impl/TestResultSetLoaderMapArray.java          |  47 +-
 .../resultSet/impl/TestResultSetLoaderMaps.java    | 107 +---
 .../impl/TestResultSetLoaderOmittedValues.java     |  28 +-
 .../impl/TestResultSetLoaderOverflow.java          |  56 +-
 .../impl/TestResultSetLoaderProjection.java        | 364 ++++++++++---
 .../impl/TestResultSetLoaderProtocol.java          |  16 +-
 .../impl/TestResultSetLoaderRepeatedList.java      |  18 +-
 .../resultSet/impl/TestResultSetLoaderTorture.java |   6 +-
 .../impl/TestResultSetLoaderTypeConversion.java    | 223 --------
 .../resultSet/impl/TestResultSetLoaderUnions.java  |  14 +-
 .../resultSet/impl/TestResultSetReader.java        |   6 +-
 .../resultSet/impl/TestResultSetSchemaChange.java  |   4 +-
 .../resultSet/impl/TestResultVectorCache.java      |  36 +-
 .../resultSet/project/TestProjectedPath.java       |  28 +-
 ...rojectedTuple.java => TestTupleProjection.java} |  41 +-
 .../impl => rowSet}/RowSetTestUtils.java           |  13 +-
 .../exec/physical/rowSet/TestDummyWriter.java      |  18 +-
 .../apache/drill/exec/record/vector/TestLoad.java  |   6 +-
 .../store/easy/text/compliant/BaseCsvTest.java     |   1 -
 .../text/compliant/TestCsvTableProperties.java     |  67 ++-
 .../compliant/TextCsvWithoutHeadersWithSchema.java | 163 ++++++
 .../apache/drill/exec/store/log/TestLogReader.java |  58 +-
 .../apache/drill/test/rowSet/RowSetComparison.java |  15 +-
 .../apache/drill/test/rowSet/RowSetUtilities.java  |   4 +-
 .../drill/test/rowSet/test/PerformanceTool.java    |   2 +-
 .../exec/record/metadata/AbstractPropertied.java   |   3 +-
 .../drill/exec/record/metadata/ColumnMetadata.java |  14 +
 .../drill/exec/record/metadata/MetadataUtils.java  |  22 +-
 .../record/metadata/PrimitiveColumnMetadata.java   |  15 +-
 .../drill/exec/record/metadata/Propertied.java     |   3 +-
 .../drill/exec/vector/accessor/ScalarWriter.java   |  39 +-
 .../drill/exec/vector/accessor/TupleWriter.java    |   7 +-
 .../{ScalarWriter.java => ValueWriter.java}        |  59 +-
 .../accessor/convert/ColumnConversionFactory.java  |  44 --
 .../accessor/convert/StandardConversions.java      | 369 -------------
 .../exec/vector/accessor/convert/package-info.java |  44 --
 .../vector/accessor/reader/BaseScalarReader.java   |   8 -
 .../accessor/writer/AbstractFixedWidthWriter.java  |  26 +-
 .../accessor/writer/AbstractObjectWriter.java      |  16 +-
 .../accessor/writer/AbstractScalarWriter.java      |  10 +-
 .../accessor/writer/AbstractScalarWriterImpl.java  |  16 +-
 .../vector/accessor/writer/BaseScalarWriter.java   |   8 -
 .../accessor/writer/ColumnWriterFactory.java       |  96 ++--
 .../vector/accessor/writer/DictEntryWriter.java    |   9 +-
 .../accessor/writer/NullableScalarWriter.java      |   7 +-
 .../vector/accessor/writer/ScalarArrayWriter.java  |  25 +-
 .../vector/accessor/writer/UnionVectorShim.java    |   2 +-
 192 files changed, 4060 insertions(+), 5213 deletions(-)

diff --git a/contrib/format-esri/src/main/java/org/apache/drill/exec/store/esri/ShpBatchReader.java b/contrib/format-esri/src/main/java/org/apache/drill/exec/store/esri/ShpBatchReader.java
index 82c5896..18cc022 100644
--- a/contrib/format-esri/src/main/java/org/apache/drill/exec/store/esri/ShpBatchReader.java
+++ b/contrib/format-esri/src/main/java/org/apache/drill/exec/store/esri/ShpBatchReader.java
@@ -59,7 +59,7 @@ public class ShpBatchReader implements ManagedReader<FileSchemaNegotiator> {
 
   private FileSplit split;
   private ResultSetLoader loader;
-  private ShpReaderConfig readerConfig;
+  private final ShpReaderConfig readerConfig;
   private Path hadoopShp;
   private Path hadoopDbf;
   private Path hadoopPrj;
@@ -104,7 +104,7 @@ public class ShpBatchReader implements ManagedReader<FileSchemaNegotiator> {
       .addNullable(SHAPE_TYPE_FIELD_NAME, TypeProtos.MinorType.VARCHAR)
       .addNullable(GEOM_FIELD_NAME, TypeProtos.MinorType.VARBINARY);
 
-    negotiator.setTableSchema(builder.buildSchema(), false);
+    negotiator.tableSchema(builder.buildSchema(), false);
     loader = negotiator.build();
 
     rowWriter = loader.writer();
@@ -185,7 +185,7 @@ public class ShpBatchReader implements ManagedReader<FileSchemaNegotiator> {
     gidWriter.setInt(gid);
     sridWriter.setInt(srid);
     shapeTypeWriter.setString(geom.getType().toString());
-    final ByteBuffer buf = (ByteBuffer) OGCGeometry.createFromEsriGeometry(geom, spatialReference).asBinary();
+    final ByteBuffer buf = OGCGeometry.createFromEsriGeometry(geom, spatialReference).asBinary();
     final byte[] bytes = buf.array();
     geomWriter.setBytes(bytes, bytes.length);
     writeDbfRow(dbfRow, rowWriter);
diff --git a/contrib/format-esri/src/main/java/org/apache/drill/exec/store/esri/ShpFormatPlugin.java b/contrib/format-esri/src/main/java/org/apache/drill/exec/store/esri/ShpFormatPlugin.java
index 3461b69..49ee364 100644
--- a/contrib/format-esri/src/main/java/org/apache/drill/exec/store/esri/ShpFormatPlugin.java
+++ b/contrib/format-esri/src/main/java/org/apache/drill/exec/store/esri/ShpFormatPlugin.java
@@ -71,7 +71,7 @@ public class ShpFormatPlugin extends EasyFormatPlugin<ShpFormatConfig> {
     FileScanFramework.FileScanBuilder builder = new FileScanFramework.FileScanBuilder();
     builder.setReaderFactory(new ShpReaderFactory(new ShpReaderConfig(this)));
     initScanBuilder(builder, scan);
-    builder.setNullType(Types.optional(TypeProtos.MinorType.VARCHAR));
+    builder.nullType(Types.optional(TypeProtos.MinorType.VARCHAR));
     return builder;
   }
 
diff --git a/contrib/format-excel/src/main/java/org/apache/drill/exec/store/excel/ExcelFormatPlugin.java b/contrib/format-excel/src/main/java/org/apache/drill/exec/store/excel/ExcelFormatPlugin.java
index 448e469..47e35b2 100644
--- a/contrib/format-excel/src/main/java/org/apache/drill/exec/store/excel/ExcelFormatPlugin.java
+++ b/contrib/format-excel/src/main/java/org/apache/drill/exec/store/excel/ExcelFormatPlugin.java
@@ -93,7 +93,7 @@ public class ExcelFormatPlugin extends EasyFormatPlugin<ExcelFormatConfig> {
     builder.setReaderFactory(new ExcelReaderFactory(readerConfig));
 
     initScanBuilder(builder, scan);
-    builder.setNullType(Types.optional(TypeProtos.MinorType.VARCHAR));
+    builder.nullType(Types.optional(TypeProtos.MinorType.VARCHAR));
     return builder;
   }
 
diff --git a/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/HDF5BatchReader.java b/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/HDF5BatchReader.java
index 2ea26d3..68094f8 100644
--- a/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/HDF5BatchReader.java
+++ b/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/HDF5BatchReader.java
@@ -46,6 +46,7 @@ import org.apache.drill.exec.store.hdf5.writers.HDF5MapDataWriter;
 import org.apache.drill.exec.store.hdf5.writers.HDF5StringDataWriter;
 import org.apache.drill.exec.store.hdf5.writers.HDF5TimestampDataWriter;
 import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.store.hdf5.writers.WriterSpec;
 import org.apache.drill.exec.vector.accessor.ArrayWriter;
 import org.apache.drill.exec.vector.accessor.ScalarWriter;
 import org.apache.drill.exec.vector.accessor.TupleWriter;
@@ -120,6 +121,8 @@ public class HDF5BatchReader implements ManagedReader<FileSchemaNegotiator> {
 
   private RowSetLoader rowWriter;
 
+  private WriterSpec writerSpec;
+
   private Iterator<HDF5DrillMetadata> metadataIterator;
 
   private ScalarWriter pathWriter;
@@ -168,8 +171,8 @@ public class HDF5BatchReader implements ManagedReader<FileSchemaNegotiator> {
     } catch (IOException e) {
       throw UserException
         .dataReadError(e)
-        .message("Failed to close input file: %s", split.getPath())
-        .message(e.getMessage())
+        .addContext("Failed to close input file: %s", split.getPath())
+        .addContext(negotiator.parentErrorContext())
         .build(logger);
     }
 
@@ -189,7 +192,7 @@ public class HDF5BatchReader implements ManagedReader<FileSchemaNegotiator> {
         .addNullable(DATASET_DATA_TYPE_NAME, TypeProtos.MinorType.VARCHAR)
         .addNullable(DIMENSIONS_FIELD_NAME, TypeProtos.MinorType.VARCHAR);
 
-      negotiator.setTableSchema(builder.buildSchema(), false);
+      negotiator.tableSchema(builder.buildSchema(), false);
 
       loader = negotiator.build();
       dimensions = new long[0];
@@ -203,6 +206,8 @@ public class HDF5BatchReader implements ManagedReader<FileSchemaNegotiator> {
 
       loader = negotiator.build();
       rowWriter = loader.writer();
+      writerSpec = new WriterSpec(rowWriter, negotiator.providedSchema(),
+          negotiator.parentErrorContext());
       if (dimensions.length <= 1) {
         buildSchemaFor1DimensionalDataset(dsInfo);
       } else if (dimensions.length == 2) {
@@ -239,41 +244,41 @@ public class HDF5BatchReader implements ManagedReader<FileSchemaNegotiator> {
       logger.warn("Couldn't add {}", dsInfo.getTypeInformation().tryGetJavaType().toGenericString());
       return;
     }
+    dataWriters.add(buildWriter(currentDataType));
+  }
 
-    switch (currentDataType) {
+  private HDF5DataWriter buildWriter(TypeProtos.MinorType dataType) {
+    switch (dataType) {
       case GENERIC_OBJECT:
-        dataWriters.add(new HDF5EnumDataWriter(hdf5Reader, rowWriter, readerConfig.defaultPath));
-        break;
+        return new HDF5EnumDataWriter(hdf5Reader, writerSpec, readerConfig.defaultPath);
       case VARCHAR:
-        dataWriters.add(new HDF5StringDataWriter(hdf5Reader, rowWriter, readerConfig.defaultPath));
-        break;
+        return new HDF5StringDataWriter(hdf5Reader, writerSpec, readerConfig.defaultPath);
       case TIMESTAMP:
-        dataWriters.add(new HDF5TimestampDataWriter(hdf5Reader, rowWriter, readerConfig.defaultPath));
-        break;
+        return new HDF5TimestampDataWriter(hdf5Reader, writerSpec, readerConfig.defaultPath);
       case INT:
-        dataWriters.add(new HDF5IntDataWriter(hdf5Reader, rowWriter, readerConfig.defaultPath));
-        break;
+        return new HDF5IntDataWriter(hdf5Reader, writerSpec, readerConfig.defaultPath);
       case BIGINT:
-        dataWriters.add(new HDF5LongDataWriter(hdf5Reader, rowWriter, readerConfig.defaultPath));
-        break;
+        return new HDF5LongDataWriter(hdf5Reader, writerSpec, readerConfig.defaultPath);
       case FLOAT8:
-        dataWriters.add(new HDF5DoubleDataWriter(hdf5Reader, rowWriter, readerConfig.defaultPath));
-        break;
+        return new HDF5DoubleDataWriter(hdf5Reader, writerSpec, readerConfig.defaultPath);
       case FLOAT4:
-        dataWriters.add(new HDF5FloatDataWriter(hdf5Reader, rowWriter, readerConfig.defaultPath));
-        break;
+        return new HDF5FloatDataWriter(hdf5Reader, writerSpec, readerConfig.defaultPath);
       case MAP:
-        dataWriters.add(new HDF5MapDataWriter(hdf5Reader, rowWriter, readerConfig.defaultPath));
-        break;
+        return new HDF5MapDataWriter(hdf5Reader, writerSpec, readerConfig.defaultPath);
+       default:
+        throw new UnsupportedOperationException(dataType.name());
     }
   }
 
   /**
-   * This function builds a Drill schema from a dataset with 2 or more dimensions.  HDF5 only supports INT, LONG, DOUBLE and FLOAT for >2 data types so this function is
-   * not as inclusinve as the 1D function.  This function will build the schema by adding DataWriters to the dataWriters array.
-   * @param dsInfo The dataset which Drill will use to build a schema
+   * Builds a Drill schema from a dataset with 2 or more dimensions. HDF5 only
+   * supports INT, LONG, DOUBLE and FLOAT for >2 data types so this function is
+   * not as inclusinve as the 1D function. This function will build the schema
+   * by adding DataWriters to the dataWriters array.
+   *
+   * @param dsInfo
+   *          The dataset which Drill will use to build a schema
    */
-
   private void buildSchemaFor2DimensionalDataset(HDF5DataSetInformation dsInfo) {
     TypeProtos.MinorType currentDataType = HDF5Utils.getDataType(dsInfo);
     // Case for null or unknown data types:
@@ -288,25 +293,27 @@ public class HDF5BatchReader implements ManagedReader<FileSchemaNegotiator> {
       switch (currentDataType) {
         case INT:
           tempFieldName = INT_COLUMN_PREFIX + i;
-          dataWriters.add(new HDF5IntDataWriter(hdf5Reader, rowWriter, readerConfig.defaultPath, tempFieldName, i));
+          dataWriters.add(new HDF5IntDataWriter(hdf5Reader, writerSpec, readerConfig.defaultPath, tempFieldName, i));
           break;
         case BIGINT:
           tempFieldName = LONG_COLUMN_PREFIX + i;
-          dataWriters.add(new HDF5LongDataWriter(hdf5Reader, rowWriter, readerConfig.defaultPath, tempFieldName, i));
+          dataWriters.add(new HDF5LongDataWriter(hdf5Reader, writerSpec, readerConfig.defaultPath, tempFieldName, i));
           break;
         case FLOAT8:
           tempFieldName = DOUBLE_COLUMN_PREFIX + i;
-          dataWriters.add(new HDF5DoubleDataWriter(hdf5Reader, rowWriter, readerConfig.defaultPath, tempFieldName, i));
+          dataWriters.add(new HDF5DoubleDataWriter(hdf5Reader, writerSpec, readerConfig.defaultPath, tempFieldName, i));
           break;
         case FLOAT4:
           tempFieldName = FLOAT_COLUMN_PREFIX + i;
-          dataWriters.add(new HDF5FloatDataWriter(hdf5Reader, rowWriter, readerConfig.defaultPath, tempFieldName, i));
+          dataWriters.add(new HDF5FloatDataWriter(hdf5Reader, writerSpec, readerConfig.defaultPath, tempFieldName, i));
           break;
+        default:
+          throw new UnsupportedOperationException(currentDataType.name());
       }
     }
   }
   /**
-   * This function contains the logic to open an HDF5 file.
+   * Opens an HDF5 file.
    * @param negotiator The negotiator represents Drill's interface with the file system
    */
   private void openFile(FileSchemaNegotiator negotiator) throws IOException {
@@ -329,7 +336,7 @@ public class HDF5BatchReader implements ManagedReader<FileSchemaNegotiator> {
   }
 
   /**
-   * This function converts the Drill InputStream into a File object for the HDF5 library. This function
+   * Converts the Drill InputStream into a File object for the HDF5 library. This function
    * exists due to a known limitation in the HDF5 library which cannot parse HDF5 directly from an input stream. A future
    * release of the library will support this.
    *
@@ -372,7 +379,9 @@ public class HDF5BatchReader implements ManagedReader<FileSchemaNegotiator> {
         if (!dataWriters.get(0).hasNext()) {
           return false;
         }
+        rowWriter.start();
         dataWriters.get(0).write();
+        rowWriter.save();
       } else if (dimensions.length <= 1) {
         // Case for Compound Data Type
         if (!dataWriters.get(0).hasNext()) {
@@ -401,7 +410,7 @@ public class HDF5BatchReader implements ManagedReader<FileSchemaNegotiator> {
   }
 
   /**
-   * This function writes one row of HDF5 metadata.
+   * Writes one row of HDF5 metadata.
    * @param rowWriter The input rowWriter object
    */
   private void projectMetadataRow(RowSetLoader rowWriter) {
@@ -434,10 +443,13 @@ public class HDF5BatchReader implements ManagedReader<FileSchemaNegotiator> {
   }
 
   /**
-   * This function gets the file metadata from a given HDF5 file.  It will extract the file name the path, and adds any information to the
-   * metadata List.
-   * @param members A list of paths from which the metadata will be extracted
-   * @param metadata the HDF5 metadata object from which the metadata will be extracted
+   * Gets the file metadata from a given HDF5 file. It will extract the file
+   * name the path, and adds any information to the metadata List.
+   *
+   * @param members
+   *          A list of paths from which the metadata will be extracted
+   * @param metadata
+   *          the HDF5 metadata object from which the metadata will be extracted
    * @return A list of metadata from the given file paths
    */
   private List<HDF5DrillMetadata> getFileMetadata(List<HDF5LinkInformation> members, List<HDF5DrillMetadata> metadata) {
@@ -496,11 +508,16 @@ public class HDF5BatchReader implements ManagedReader<FileSchemaNegotiator> {
   }
 
   /**
-   * This function writes one row of data in a metadata query. The number of dimensions here is n+1. So if the actual dataset is a 1D column, it will be written as a list.
-   * This is function is only called in metadata queries as the schema is not known in advance. If the datasize is greater than 16MB, the function does not project the dataset
+   * Writes one row of data in a metadata query. The number of dimensions here
+   * is n+1. So if the actual dataset is a 1D column, it will be written as a list.
+   * This is function is only called in metadata queries as the schema is not
+   * known in advance. If the datasize is greater than 16MB, the function does
+   * not project the dataset
    *
-   * @param rowWriter The rowWriter to which the data will be written
-   * @param datapath The datapath from which the data will be read
+   * @param rowWriter
+   *          The rowWriter to which the data will be written
+   * @param datapath
+   *          The datapath from which the data will be read
    */
   private void projectDataset(RowSetLoader rowWriter, String datapath) {
     String fieldName = HDF5Utils.getNameFromPath(datapath);
@@ -697,8 +714,9 @@ public class HDF5BatchReader implements ManagedReader<FileSchemaNegotiator> {
   }
 
   private void intMatrixHelper(int[][] colData, int cols, int rows, RowSetLoader rowWriter) {
-    // This is the case where a dataset is projected in a metadata query.  The result should be a list of lists
 
+    // This is the case where a dataset is projected in a metadata query.
+    // The result should be a list of lists
     TupleMetadata nestedSchema = new SchemaBuilder()
       .addRepeatedList(INT_COLUMN_NAME)
       .addArray(TypeProtos.MinorType.INT)
@@ -971,9 +989,13 @@ public class HDF5BatchReader implements ManagedReader<FileSchemaNegotiator> {
   }
 
   /**
-   * This helper function gets the attributes for an HDF5 datapath.  These attributes are projected as a map in select * queries when the defaultPath is null.
-   * @param rowWriter the row to which the data will be written
-   * @param record the record for the attributes
+   * Gets the attributes for an HDF5 datapath. These attributes are projected as
+   * a map in select * queries when the defaultPath is null.
+   *
+   * @param rowWriter
+   *          the row to which the data will be written
+   * @param record
+   *          the record for the attributes
    */
   private void writeAttributes(TupleWriter rowWriter, HDF5DrillMetadata record) {
     Map<String, HDF5Attribute> attribs = getAttributes(record.getPath());
@@ -1022,12 +1044,14 @@ public class HDF5BatchReader implements ManagedReader<FileSchemaNegotiator> {
           String enumText = attrib.getValue().toString();
           writeStringColumn(mapWriter, key, enumText);
           break;
+        default:
+          throw new IllegalStateException(attrib.getDataType().name());
       }
     }
   }
 
   /**
-   * This function processes the MAP data type which can be found in HDF5 files.
+   * Processes the MAP data type which can be found in HDF5 files.
    * It automatically flattens anything greater than 2 dimensions.
    *
    * @param path the HDF5 path tp the compound data
@@ -1035,7 +1059,6 @@ public class HDF5BatchReader implements ManagedReader<FileSchemaNegotiator> {
    * @param reader the HDF5 reader for the data file
    * @param rowWriter the rowWriter to write the data
    */
-
   private void getAndMapCompoundData(String path, List<String> fieldNames, IHDF5Reader reader, RowSetLoader rowWriter) {
 
     final String COMPOUND_DATA_FIELD_NAME = "compound_data";
diff --git a/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/HDF5FormatPlugin.java b/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/HDF5FormatPlugin.java
index b696cdb..60d6ec5 100644
--- a/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/HDF5FormatPlugin.java
+++ b/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/HDF5FormatPlugin.java
@@ -75,7 +75,7 @@ public class HDF5FormatPlugin extends EasyFormatPlugin<HDF5FormatConfig> {
 
     builder.setReaderFactory(new HDF5ReaderFactory(new HDF5BatchReader.HDF5ReaderConfig(this, formatConfig)));
     initScanBuilder(builder, scan);
-    builder.setNullType(Types.optional(TypeProtos.MinorType.VARCHAR));
+    builder.nullType(Types.optional(TypeProtos.MinorType.VARCHAR));
     return builder;
   }
 
diff --git a/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/HDF5DataWriter.java b/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/HDF5DataWriter.java
index e86ece1..79865e5 100644
--- a/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/HDF5DataWriter.java
+++ b/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/HDF5DataWriter.java
@@ -18,20 +18,12 @@
 
 package org.apache.drill.exec.store.hdf5.writers;
 
-import ch.systemsx.cisd.hdf5.IHDF5Reader;
-import org.apache.drill.common.types.TypeProtos;
-import org.apache.drill.exec.physical.resultSet.RowSetLoader;
-import org.apache.drill.exec.record.metadata.ColumnMetadata;
-import org.apache.drill.exec.record.metadata.MetadataUtils;
-import org.apache.drill.exec.vector.accessor.ScalarWriter;
-import org.apache.drill.exec.vector.accessor.TupleWriter;
-
 import java.util.ArrayList;
 import java.util.List;
 
-public abstract class HDF5DataWriter {
-  protected final RowSetLoader columnWriter;
+import ch.systemsx.cisd.hdf5.IHDF5Reader;
 
+public abstract class HDF5DataWriter {
   protected final IHDF5Reader reader;
 
   protected final String datapath;
@@ -44,16 +36,13 @@ public abstract class HDF5DataWriter {
 
   protected Object[][] compoundData;
 
-  public HDF5DataWriter(IHDF5Reader reader, RowSetLoader columnWriter, String datapath) {
+  public HDF5DataWriter(IHDF5Reader reader, String datapath) {
     this.reader = reader;
-    this.columnWriter = columnWriter;
     this.datapath = datapath;
   }
 
-  public HDF5DataWriter(IHDF5Reader reader, RowSetLoader columnWriter, String datapath, String fieldName, int colCount) {
-    this.reader = reader;
-    this.columnWriter = columnWriter;
-    this.datapath = datapath;
+  public HDF5DataWriter(IHDF5Reader reader, String datapath, String fieldName, int colCount) {
+    this(reader, datapath);
     this.fieldName = fieldName;
     this.colCount = colCount;
   }
@@ -70,10 +59,11 @@ public abstract class HDF5DataWriter {
     return counter;
   }
 
-  public List<Object> getColumn(int columnIndex) {
-    List<Object> result = new ArrayList<>();
+  @SuppressWarnings("unchecked")
+  public <T> List<T> getColumn(int columnIndex) {
+    List<T> result = new ArrayList<>();
     for (Object[] compoundDatum : compoundData) {
-      result.add(compoundDatum[columnIndex]);
+      result.add((T) compoundDatum[columnIndex]);
     }
     return result;
   }
@@ -83,11 +73,4 @@ public abstract class HDF5DataWriter {
   public boolean isCompound() {
     return false;
   }
-
-  protected static ScalarWriter makeWriter(TupleWriter tupleWriter, String name, TypeProtos.MinorType type, TypeProtos.DataMode mode) {
-    ColumnMetadata colSchema = MetadataUtils.newScalar(name, type, mode);
-    int index = tupleWriter.addColumn(colSchema);
-    return tupleWriter.scalar(index);
-  }
-
 }
diff --git a/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/HDF5DoubleDataWriter.java b/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/HDF5DoubleDataWriter.java
index 5fcfa7c..0143854 100644
--- a/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/HDF5DoubleDataWriter.java
+++ b/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/HDF5DoubleDataWriter.java
@@ -18,32 +18,32 @@
 
 package org.apache.drill.exec.store.hdf5.writers;
 
-import ch.systemsx.cisd.hdf5.IHDF5Reader;
+import java.util.List;
+
 import org.apache.drill.common.types.TypeProtos;
-import org.apache.drill.exec.physical.resultSet.RowSetLoader;
 import org.apache.drill.exec.store.hdf5.HDF5Utils;
-import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.ValueWriter;
 
-import java.util.List;
+import ch.systemsx.cisd.hdf5.IHDF5Reader;
 
 public class HDF5DoubleDataWriter extends HDF5DataWriter {
 
   private final double[] data;
 
-  private final ScalarWriter rowWriter;
+  private final ValueWriter colWriter;
 
   // This constructor is used when the data is a 1D column.  The column is inferred from the datapath
-  public HDF5DoubleDataWriter(IHDF5Reader reader, RowSetLoader columnWriter, String datapath) {
-    super(reader, columnWriter, datapath);
+  public HDF5DoubleDataWriter(IHDF5Reader reader, WriterSpec writerSpec, String datapath) {
+    super(reader, datapath);
     data = reader.readDoubleArray(datapath);
 
     fieldName = HDF5Utils.getNameFromPath(datapath);
-    rowWriter = makeWriter(columnWriter, fieldName, TypeProtos.MinorType.FLOAT8, TypeProtos.DataMode.OPTIONAL);
+    colWriter = writerSpec.makeWriter(fieldName, TypeProtos.MinorType.FLOAT8, TypeProtos.DataMode.OPTIONAL);
   }
 
   // This constructor is used when the data is part of a 2D array.  In this case the column name is provided in the constructor
-  public HDF5DoubleDataWriter(IHDF5Reader reader, RowSetLoader columnWriter, String datapath, String fieldName, int currentColumn) {
-    super(reader, columnWriter, datapath, fieldName, currentColumn);
+  public HDF5DoubleDataWriter(IHDF5Reader reader, WriterSpec writerSpec, String datapath, String fieldName, int currentColumn) {
+    super(reader, datapath, fieldName, currentColumn);
     // Get dimensions
     long[] dimensions = reader.object().getDataSetInformation(datapath).getDimensions();
     double[][] tempData;
@@ -53,29 +53,30 @@ public class HDF5DoubleDataWriter extends HDF5DataWriter {
       tempData = transpose(reader.float64().readMDArray(datapath).toMatrix());
     }
     data = tempData[currentColumn];
-    rowWriter = makeWriter(columnWriter, fieldName, TypeProtos.MinorType.FLOAT8, TypeProtos.DataMode.OPTIONAL);
+    colWriter = writerSpec.makeWriter(fieldName, TypeProtos.MinorType.FLOAT8, TypeProtos.DataMode.OPTIONAL);
   }
 
-  public HDF5DoubleDataWriter(IHDF5Reader reader, RowSetLoader columnWriter, String fieldName, List<Double> tempListData) {
-    super(reader, columnWriter, null);
+  public HDF5DoubleDataWriter(IHDF5Reader reader, WriterSpec writerSpec, String fieldName, List<Double> tempListData) {
+    super(reader, null);
     this.fieldName = fieldName;
     data = new double[tempListData.size()];
     for (int i = 0; i < tempListData.size(); i++) {
       data[i] = tempListData.get(i);
     }
-    rowWriter = makeWriter(columnWriter, fieldName, TypeProtos.MinorType.FLOAT8, TypeProtos.DataMode.OPTIONAL);
+    colWriter = writerSpec.makeWriter(fieldName, TypeProtos.MinorType.FLOAT8, TypeProtos.DataMode.OPTIONAL);
   }
 
-
+  @Override
   public boolean write() {
     if (counter > data.length) {
       return false;
     } else {
-      rowWriter.setDouble(data[counter++]);
+      colWriter.setDouble(data[counter++]);
       return true;
     }
   }
 
+  @Override
   public boolean hasNext() {
     return counter < data.length;
   }
@@ -103,6 +104,7 @@ public class HDF5DoubleDataWriter extends HDF5DataWriter {
     return transposedMatrix;
   }
 
+  @Override
   public int getDataSize() {
     return data.length;
   }
diff --git a/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/HDF5EnumDataWriter.java b/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/HDF5EnumDataWriter.java
index 7942c75..136f454 100644
--- a/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/HDF5EnumDataWriter.java
+++ b/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/HDF5EnumDataWriter.java
@@ -18,32 +18,33 @@
 
 package org.apache.drill.exec.store.hdf5.writers;
 
-import ch.systemsx.cisd.hdf5.IHDF5Reader;
 import org.apache.drill.common.types.TypeProtos;
-import org.apache.drill.exec.physical.resultSet.RowSetLoader;
 import org.apache.drill.exec.store.hdf5.HDF5Utils;
-import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.ValueWriter;
+
+import ch.systemsx.cisd.hdf5.IHDF5Reader;
 
 public class HDF5EnumDataWriter extends HDF5DataWriter {
 
   private final String[] data;
 
-  private final ScalarWriter rowWriter;
+  private final ValueWriter colWriter;
 
   // This constructor is used when the data is a 1D column.  The column is inferred from the datapath
-  public HDF5EnumDataWriter(IHDF5Reader reader, RowSetLoader columnWriter, String datapath) {
-    super(reader, columnWriter, datapath);
+  public HDF5EnumDataWriter(IHDF5Reader reader, WriterSpec writerSpec, String datapath) {
+    super(reader, datapath);
     data = reader.readEnumArrayAsString(datapath);
 
     fieldName = HDF5Utils.getNameFromPath(datapath);
-    rowWriter = makeWriter(columnWriter, fieldName, TypeProtos.MinorType.VARCHAR, TypeProtos.DataMode.OPTIONAL);
+    colWriter = writerSpec.makeWriter(fieldName, TypeProtos.MinorType.VARCHAR, TypeProtos.DataMode.OPTIONAL);
   }
 
+  @Override
   public boolean write() {
     if (counter > data.length) {
       return false;
     } else {
-      rowWriter.setString(data[counter++]);
+      colWriter.setString(data[counter++]);
       return true;
     }
   }
@@ -51,6 +52,7 @@ public class HDF5EnumDataWriter extends HDF5DataWriter {
   @Override
   public int getDataSize() { return data.length; }
 
+  @Override
   public boolean hasNext() {
     return counter < data.length;
   }
diff --git a/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/HDF5FloatDataWriter.java b/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/HDF5FloatDataWriter.java
index 69ec164..5cd20e2 100644
--- a/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/HDF5FloatDataWriter.java
+++ b/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/HDF5FloatDataWriter.java
@@ -18,31 +18,31 @@
 
 package org.apache.drill.exec.store.hdf5.writers;
 
-import ch.systemsx.cisd.hdf5.IHDF5Reader;
+import java.util.List;
+
 import org.apache.drill.common.types.TypeProtos;
-import org.apache.drill.exec.physical.resultSet.RowSetLoader;
 import org.apache.drill.exec.store.hdf5.HDF5Utils;
-import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.ValueWriter;
 
-import java.util.List;
+import ch.systemsx.cisd.hdf5.IHDF5Reader;
 
 public class HDF5FloatDataWriter extends HDF5DataWriter {
 
   private final float[] data;
 
-  private final ScalarWriter rowWriter;
+  private final ValueWriter colWriter;
 
   // This constructor is used when the data is a 1D column.  The column is inferred from the datapath
-  public HDF5FloatDataWriter(IHDF5Reader reader, RowSetLoader columnWriter, String datapath) {
-    super(reader, columnWriter, datapath);
+  public HDF5FloatDataWriter(IHDF5Reader reader, WriterSpec writerSpec, String datapath) {
+    super(reader, datapath);
     data = reader.readFloatArray(datapath);
     fieldName = HDF5Utils.getNameFromPath(datapath);
-    rowWriter = makeWriter(columnWriter, fieldName, TypeProtos.MinorType.FLOAT8, TypeProtos.DataMode.OPTIONAL);
+    colWriter = writerSpec.makeWriter(fieldName, TypeProtos.MinorType.FLOAT8, TypeProtos.DataMode.OPTIONAL);
   }
 
   // This constructor is used when the data is part of a 2D array.  In this case the column name is provided in the constructor
-  public HDF5FloatDataWriter(IHDF5Reader reader, RowSetLoader columnWriter, String datapath, String fieldName, int currentColumn) {
-    super(reader, columnWriter, datapath, fieldName, currentColumn);
+  public HDF5FloatDataWriter(IHDF5Reader reader, WriterSpec writerSpec, String datapath, String fieldName, int currentColumn) {
+    super(reader, datapath, fieldName, currentColumn);
     // Get dimensions
     long[] dimensions = reader.object().getDataSetInformation(datapath).getDimensions();
     float[][] tempData;
@@ -52,29 +52,31 @@ public class HDF5FloatDataWriter extends HDF5DataWriter {
       tempData = transpose(reader.float32().readMDArray(datapath).toMatrix());
     }
     data = tempData[currentColumn];
-    rowWriter = makeWriter(columnWriter, fieldName, TypeProtos.MinorType.FLOAT8, TypeProtos.DataMode.OPTIONAL);
+    colWriter = writerSpec.makeWriter(fieldName, TypeProtos.MinorType.FLOAT8, TypeProtos.DataMode.OPTIONAL);
   }
 
-  public HDF5FloatDataWriter(IHDF5Reader reader, RowSetLoader columnWriter, String fieldName, List<Float> tempListData) {
-    super(reader, columnWriter, null);
+  public HDF5FloatDataWriter(IHDF5Reader reader, WriterSpec writerSpec, String fieldName, List<Float> tempListData) {
+    super(reader, null);
     this.fieldName = fieldName;
     data = new float[tempListData.size()];
     for (int i = 0; i < tempListData.size(); i++) {
       data[i] = tempListData.get(i);
     }
-    rowWriter = makeWriter(columnWriter, fieldName, TypeProtos.MinorType.FLOAT8, TypeProtos.DataMode.OPTIONAL);
+    colWriter = writerSpec.makeWriter(fieldName, TypeProtos.MinorType.FLOAT8, TypeProtos.DataMode.OPTIONAL);
   }
 
 
+  @Override
   public boolean write() {
     if (counter > data.length) {
       return false;
     } else {
-      rowWriter.setDouble(data[counter++]);
+      colWriter.setDouble(data[counter++]);
       return true;
     }
   }
 
+  @Override
   public boolean hasNext() {
     return counter < data.length;
   }
diff --git a/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/HDF5IntDataWriter.java b/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/HDF5IntDataWriter.java
index 221acb8..e2cea35 100644
--- a/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/HDF5IntDataWriter.java
+++ b/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/HDF5IntDataWriter.java
@@ -18,32 +18,33 @@
 
 package org.apache.drill.exec.store.hdf5.writers;
 
-import ch.systemsx.cisd.hdf5.IHDF5Reader;
+import java.util.List;
+
 import org.apache.drill.common.types.TypeProtos;
-import org.apache.drill.exec.physical.resultSet.RowSetLoader;
 import org.apache.drill.exec.store.hdf5.HDF5Utils;
-import org.apache.drill.exec.vector.accessor.ScalarWriter;
-import java.util.List;
+import org.apache.drill.exec.vector.accessor.ValueWriter;
+
+import ch.systemsx.cisd.hdf5.IHDF5Reader;
 
 public class HDF5IntDataWriter extends HDF5DataWriter {
 
   private final int[] data;
 
-  private final ScalarWriter rowWriter;
+  private final ValueWriter colWriter;
 
   // This constructor is used when the data is a 1D column.  The column is inferred from the datapath
-  public HDF5IntDataWriter(IHDF5Reader reader, RowSetLoader columnWriter, String datapath) {
-    super(reader, columnWriter, datapath);
+  public HDF5IntDataWriter(IHDF5Reader reader, WriterSpec writerSpec, String datapath) {
+    super(reader, datapath);
     data = reader.readIntArray(datapath);
 
     fieldName = HDF5Utils.getNameFromPath(datapath);
 
-    rowWriter = makeWriter(columnWriter, fieldName, TypeProtos.MinorType.INT, TypeProtos.DataMode.OPTIONAL);
+    colWriter = writerSpec.makeWriter(fieldName, TypeProtos.MinorType.INT, TypeProtos.DataMode.OPTIONAL);
   }
 
   // This constructor is used when the data is part of a 2D array.  In this case the column name is provided in the constructor
-  public HDF5IntDataWriter(IHDF5Reader reader, RowSetLoader columnWriter, String datapath, String fieldName, int currentColumn) {
-    super(reader, columnWriter, datapath, fieldName, currentColumn);
+  public HDF5IntDataWriter(IHDF5Reader reader, WriterSpec writerSpec, String datapath, String fieldName, int currentColumn) {
+    super(reader, datapath, fieldName, currentColumn);
     // Get dimensions
     long[] dimensions = reader.object().getDataSetInformation(datapath).getDimensions();
     int[][] tempData;
@@ -54,31 +55,32 @@ public class HDF5IntDataWriter extends HDF5DataWriter {
     }
     data = tempData[currentColumn];
 
-    rowWriter = makeWriter(columnWriter, fieldName, TypeProtos.MinorType.INT, TypeProtos.DataMode.OPTIONAL);
+    colWriter = writerSpec.makeWriter(fieldName, TypeProtos.MinorType.INT, TypeProtos.DataMode.OPTIONAL);
   }
 
   // This constructor is used for compound data types.
-  public HDF5IntDataWriter(IHDF5Reader reader, RowSetLoader columnWriter, String fieldName, List<Integer> tempListData) {
-    super(reader, columnWriter, null);
+  public HDF5IntDataWriter(IHDF5Reader reader, WriterSpec writerSpec, String fieldName, List<Integer> tempListData) {
+    super(reader, null);
     this.fieldName = fieldName;
     data = new int[tempListData.size()];
     for (int i = 0; i < tempListData.size(); i++) {
       data[i] = tempListData.get(i);
     }
 
-    rowWriter = makeWriter(columnWriter, fieldName, TypeProtos.MinorType.INT, TypeProtos.DataMode.OPTIONAL);
+    colWriter = writerSpec.makeWriter(fieldName, TypeProtos.MinorType.INT, TypeProtos.DataMode.OPTIONAL);
   }
 
-
+  @Override
   public boolean write() {
     if (counter > data.length) {
       return false;
     } else {
-      rowWriter.setInt(data[counter++]);
+      colWriter.setInt(data[counter++]);
       return true;
     }
   }
 
+  @Override
   public boolean hasNext() {
     return counter < data.length;
   }
@@ -106,6 +108,7 @@ public class HDF5IntDataWriter extends HDF5DataWriter {
     return transposedMatrix;
   }
 
+  @Override
   public int getDataSize() {
     return data.length;
   }
diff --git a/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/HDF5LongDataWriter.java b/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/HDF5LongDataWriter.java
index fe9b05a..b9f63b4 100644
--- a/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/HDF5LongDataWriter.java
+++ b/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/HDF5LongDataWriter.java
@@ -18,33 +18,32 @@
 
 package org.apache.drill.exec.store.hdf5.writers;
 
-import ch.systemsx.cisd.hdf5.IHDF5Reader;
+import java.util.List;
+
 import org.apache.drill.common.types.TypeProtos;
-import org.apache.drill.exec.physical.resultSet.RowSetLoader;
 import org.apache.drill.exec.store.hdf5.HDF5Utils;
-import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.ValueWriter;
 
-import java.util.List;
+import ch.systemsx.cisd.hdf5.IHDF5Reader;
 
 public class HDF5LongDataWriter extends HDF5DataWriter {
 
   private final long[] data;
 
-  private final ScalarWriter rowWriter;
+  private final ValueWriter colWriter;
 
   // This constructor is used when the data is a 1D column.  The column is inferred from the datapath
-  public HDF5LongDataWriter(IHDF5Reader reader, RowSetLoader columnWriter, String datapath) {
-    super(reader, columnWriter, datapath);
+  public HDF5LongDataWriter(IHDF5Reader reader, WriterSpec writerSpec, String datapath) {
+    super(reader, datapath);
     data = reader.readLongArray(datapath);
 
     fieldName = HDF5Utils.getNameFromPath(datapath);
-    rowWriter = makeWriter(columnWriter, fieldName, TypeProtos.MinorType.BIGINT, TypeProtos.DataMode.OPTIONAL);
-
+    colWriter = writerSpec.makeWriter(fieldName, TypeProtos.MinorType.BIGINT, TypeProtos.DataMode.OPTIONAL);
   }
 
   // This constructor is used when the data is part of a 2D array.  In this case the column name is provided in the constructor
-  public HDF5LongDataWriter(IHDF5Reader reader, RowSetLoader columnWriter, String datapath, String fieldName, int currentColumn) {
-    super(reader, columnWriter, datapath, fieldName, currentColumn);
+  public HDF5LongDataWriter(IHDF5Reader reader, WriterSpec writerSpec, String datapath, String fieldName, int currentColumn) {
+    super(reader, datapath, fieldName, currentColumn);
     // Get dimensions
     long[] dimensions = reader.object().getDataSetInformation(datapath).getDimensions();
     long[][] tempData;
@@ -54,28 +53,30 @@ public class HDF5LongDataWriter extends HDF5DataWriter {
       tempData = transpose(reader.int64().readMDArray(datapath).toMatrix());
     }
     data = tempData[currentColumn];
-    rowWriter = makeWriter(columnWriter, fieldName, TypeProtos.MinorType.BIGINT, TypeProtos.DataMode.OPTIONAL);
+    colWriter = writerSpec.makeWriter(fieldName, TypeProtos.MinorType.BIGINT, TypeProtos.DataMode.OPTIONAL);
   }
 
-  public HDF5LongDataWriter(IHDF5Reader reader, RowSetLoader columnWriter, String fieldName, List<Long> tempListData) {
-    super(reader, columnWriter, null);
+  public HDF5LongDataWriter(IHDF5Reader reader, WriterSpec writerSpec, String fieldName, List<Long> tempListData) {
+    super(reader, null);
     this.fieldName = fieldName;
     data = new long[tempListData.size()];
     for (int i = 0; i < tempListData.size(); i++) {
       data[i] = tempListData.get(i);
     }
-    rowWriter = makeWriter(columnWriter, fieldName, TypeProtos.MinorType.BIGINT, TypeProtos.DataMode.OPTIONAL);
+    colWriter = writerSpec.makeWriter(fieldName, TypeProtos.MinorType.BIGINT, TypeProtos.DataMode.OPTIONAL);
   }
 
+  @Override
   public boolean write() {
     if (counter > data.length) {
       return false;
     } else {
-      rowWriter.setLong(data[counter++]);
+      colWriter.setLong(data[counter++]);
       return true;
     }
   }
 
+  @Override
   public boolean hasNext() {
     return counter < data.length;
   }
@@ -103,6 +104,7 @@ public class HDF5LongDataWriter extends HDF5DataWriter {
     return transposedMatrix;
   }
 
+  @Override
   public int getDataSize() {
     return data.length;
   }
diff --git a/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/HDF5MapDataWriter.java b/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/HDF5MapDataWriter.java
index 1d34597..a0c19e2 100644
--- a/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/HDF5MapDataWriter.java
+++ b/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/HDF5MapDataWriter.java
@@ -21,7 +21,6 @@ package org.apache.drill.exec.store.hdf5.writers;
 import ch.systemsx.cisd.hdf5.HDF5CompoundMemberInformation;
 import ch.systemsx.cisd.hdf5.IHDF5Reader;
 import org.apache.drill.common.exceptions.UserException;
-import org.apache.drill.exec.physical.resultSet.RowSetLoader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -31,47 +30,47 @@ import java.util.List;
 public class HDF5MapDataWriter extends HDF5DataWriter {
   private static final Logger logger = LoggerFactory.getLogger(HDF5MapDataWriter.class);
 
-  private final String UNSAFE_SPACE_SEPARATOR = " ";
+  private static final String UNSAFE_SPACE_SEPARATOR = " ";
 
-  private final String SAFE_SPACE_SEPARATOR = "_";
+  private static final String SAFE_SPACE_SEPARATOR = "_";
 
   private final List<HDF5DataWriter> dataWriters;
 
   private List<String> fieldNames;
 
-
-  public HDF5MapDataWriter(IHDF5Reader reader, RowSetLoader columnWriter, String datapath) {
-    super(reader, columnWriter, datapath);
+  public HDF5MapDataWriter(IHDF5Reader reader, WriterSpec writerSpec, String datapath) {
+    super(reader, datapath);
     fieldNames = new ArrayList<>();
 
     compoundData = reader.compound().readArray(datapath, Object[].class);
     dataWriters = new ArrayList<>();
     fieldNames = getFieldNames();
     try {
-      getDataWriters();
+      getDataWriters(writerSpec);
     } catch (Exception e) {
       throw UserException
-        .dataReadError()
-        .message("Error writing Compound Field: %s", e.getMessage())
+        .dataReadError(e)
+        .addContext("Error writing compound field", datapath)
+        .addContext(writerSpec.errorContext)
         .build(logger);
     }
   }
 
+  @Override
   public boolean write() {
-    if (!hasNext()) {
-      return false;
-    } else {
+    if (hasNext()) {
       // Loop through the columns and write the columns
-      columnWriter.start();
       for (HDF5DataWriter dataWriter : dataWriters) {
         dataWriter.write();
       }
-      columnWriter.save();
       counter++;
       return true;
+    } else {
+      return false;
     }
   }
 
+  @Override
   public boolean hasNext() {
     return counter < dataWriters.get(0).getDataSize();
   }
@@ -87,34 +86,33 @@ public class HDF5MapDataWriter extends HDF5DataWriter {
   }
 
   /**
-   * This function will populate the ArrayList of DataWriters. Since HDF5 Maps contain homogeneous columns,
-   * it is fine to get the first row, and iterate through the columns to get the data types and build the schema accordingly.
+   * Populates the ArrayList of DataWriters. Since HDF5 Maps contain homogeneous
+   * columns, it is fine to get the first row, and iterate through the columns
+   * to get the data types and build the schema accordingly.
    */
-  private void getDataWriters() {
-    List listData;
+  private void getDataWriters(WriterSpec writerSpec) {
 
     for (int col = 0; col < compoundData[0].length; col++) {
       Object currentColumn = compoundData[0][col];
       String dataType = currentColumn.getClass().getSimpleName();
-      listData = getColumn(col);
 
       switch (dataType) {
         case "Byte":
         case "Short":
         case "Integer":
-          dataWriters.add(new HDF5IntDataWriter(reader, columnWriter, fieldNames.get(col), listData));
+          dataWriters.add(new HDF5IntDataWriter(reader, writerSpec, fieldNames.get(col), getColumn(col)));
           break;
         case "Long":
-          dataWriters.add(new HDF5LongDataWriter(reader, columnWriter, fieldNames.get(col), listData));
+          dataWriters.add(new HDF5LongDataWriter(reader, writerSpec, fieldNames.get(col), getColumn(col)));
           break;
         case "Double":
-          dataWriters.add(new HDF5DoubleDataWriter(reader, columnWriter, fieldNames.get(col), listData));
+          dataWriters.add(new HDF5DoubleDataWriter(reader, writerSpec, fieldNames.get(col), getColumn(col)));
           break;
         case "Float":
-          dataWriters.add(new HDF5FloatDataWriter(reader, columnWriter, fieldNames.get(col), listData));
+          dataWriters.add(new HDF5FloatDataWriter(reader, writerSpec, fieldNames.get(col), getColumn(col)));
           break;
         case "String":
-          dataWriters.add(new HDF5StringDataWriter(reader, columnWriter, fieldNames.get(col), listData));
+          dataWriters.add(new HDF5StringDataWriter(reader, writerSpec, fieldNames.get(col), getColumn(col)));
           break;
         default:
           // Log unknown data type
@@ -125,9 +123,10 @@ public class HDF5MapDataWriter extends HDF5DataWriter {
   }
 
   /**
-   * This function returns true if the data writer is a compound writer, false if not.
+   * Returns true if the data writer is a compound writer, false if not.
    * @return boolean true if the data writer is a compound writer, false if not.
    */
+  @Override
   public boolean isCompound() {
     return true;
   }
diff --git a/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/HDF5StringDataWriter.java b/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/HDF5StringDataWriter.java
index 6414c24..db7ea32 100644
--- a/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/HDF5StringDataWriter.java
+++ b/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/HDF5StringDataWriter.java
@@ -18,14 +18,14 @@
 
 package org.apache.drill.exec.store.hdf5.writers;
 
-import ch.systemsx.cisd.hdf5.IHDF5Reader;
+import java.util.Arrays;
+import java.util.List;
+
 import org.apache.drill.common.types.TypeProtos;
-import org.apache.drill.exec.physical.resultSet.RowSetLoader;
 import org.apache.drill.exec.store.hdf5.HDF5Utils;
-import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.ValueWriter;
 
-import java.util.Arrays;
-import java.util.List;
+import ch.systemsx.cisd.hdf5.IHDF5Reader;
 
 public class HDF5StringDataWriter extends HDF5DataWriter {
 
@@ -33,33 +33,35 @@ public class HDF5StringDataWriter extends HDF5DataWriter {
 
   private final List<String> listData;
 
-  private final ScalarWriter rowWriter;
+  private final ValueWriter colWriter;
 
   // This constructor is used when the data is a 1D column.  The column is inferred from the datapath
-  public HDF5StringDataWriter(IHDF5Reader reader, RowSetLoader columnWriter, String datapath) {
-    super(reader, columnWriter, datapath);
+  public HDF5StringDataWriter(IHDF5Reader reader, WriterSpec writerSpec, String datapath) {
+    super(reader, datapath);
     data = reader.readStringArray(datapath);
     listData = Arrays.asList(data);
     fieldName = HDF5Utils.getNameFromPath(datapath);
-    rowWriter = makeWriter(columnWriter, fieldName, TypeProtos.MinorType.VARCHAR, TypeProtos.DataMode.OPTIONAL);
+    colWriter = writerSpec.makeWriter(fieldName, TypeProtos.MinorType.VARCHAR, TypeProtos.DataMode.OPTIONAL);
   }
 
-  public HDF5StringDataWriter(IHDF5Reader reader, RowSetLoader columnWriter, String fieldName, List<String> data) {
-    super(reader, columnWriter, null);
+  public HDF5StringDataWriter(IHDF5Reader reader, WriterSpec writerSpec, String fieldName, List<String> data) {
+    super(reader, null);
     this.fieldName = fieldName;
     this.listData = data;
-    rowWriter = makeWriter(columnWriter, fieldName, TypeProtos.MinorType.VARCHAR, TypeProtos.DataMode.OPTIONAL);
+    colWriter = writerSpec.makeWriter(fieldName, TypeProtos.MinorType.VARCHAR, TypeProtos.DataMode.OPTIONAL);
   }
 
+  @Override
   public boolean write() {
     if (counter > listData.size()) {
       return false;
     } else {
-      rowWriter.setString(listData.get(counter++));
+      colWriter.setString(listData.get(counter++));
       return true;
     }
   }
 
+  @Override
   public boolean hasNext() {
     return counter < data.length;
   }
diff --git a/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/HDF5TimestampDataWriter.java b/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/HDF5TimestampDataWriter.java
index c342221..808e89f 100644
--- a/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/HDF5TimestampDataWriter.java
+++ b/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/HDF5TimestampDataWriter.java
@@ -18,37 +18,39 @@
 
 package org.apache.drill.exec.store.hdf5.writers;
 
-import ch.systemsx.cisd.hdf5.IHDF5Reader;
 import org.apache.drill.common.types.TypeProtos;
-import org.apache.drill.exec.physical.resultSet.RowSetLoader;
 import org.apache.drill.exec.store.hdf5.HDF5Utils;
-import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.ValueWriter;
 import org.joda.time.Instant;
 
+import ch.systemsx.cisd.hdf5.IHDF5Reader;
+
 public class HDF5TimestampDataWriter extends HDF5DataWriter {
 
   private final long[] data;
 
-  private final ScalarWriter rowWriter;
+  private final ValueWriter colWriter;
 
   // This constructor is used when the data is a 1D column.  The column is inferred from the datapath
-  public HDF5TimestampDataWriter(IHDF5Reader reader, RowSetLoader columnWriter, String datapath) {
-    super(reader, columnWriter, datapath);
+  public HDF5TimestampDataWriter(IHDF5Reader reader, WriterSpec writerSpec, String datapath) {
+    super(reader, datapath);
     data = reader.time().readTimeStampArray(datapath);
 
     fieldName = HDF5Utils.getNameFromPath(datapath);
-    rowWriter = makeWriter(columnWriter, fieldName, TypeProtos.MinorType.TIMESTAMP, TypeProtos.DataMode.OPTIONAL);
+    colWriter = writerSpec.makeWriter(fieldName, TypeProtos.MinorType.TIMESTAMP, TypeProtos.DataMode.OPTIONAL);
   }
 
+  @Override
   public boolean write() {
     if (counter > data.length) {
       return false;
     } else {
-      rowWriter.setTimestamp(new Instant(data[counter++]));
+      colWriter.setTimestamp(new Instant(data[counter++]));
       return true;
     }
   }
 
+  @Override
   public boolean hasNext() {
     return counter < data.length;
   }
diff --git a/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/WriterSpec.java b/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/WriterSpec.java
new file mode 100644
index 0000000..6d45e06
--- /dev/null
+++ b/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/writers/WriterSpec.java
@@ -0,0 +1,62 @@
+/*
+ * 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.hdf5.writers;
+
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.convert.StandardConversions;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.drill.exec.vector.accessor.ValueWriter;
+
+/**
+ * Encapsulates the information needed to handle implicit type conversions
+ * for scalar fields. The provided schema gives the type of vector to
+ * create, the {@link #makeWriter(String, MinorType, DataMode)} method
+ * gives the type of data to be written. A standard conversion is inserted
+ * where needed.
+ */
+public class WriterSpec {
+  public final TupleWriter tupleWriter;
+  public final TupleMetadata providedSchema;
+  public final CustomErrorContext errorContext;
+  public final StandardConversions conversions;
+
+  public WriterSpec(TupleWriter tupleWriter, TupleMetadata providedSchema, CustomErrorContext errorContext) {
+    this.tupleWriter = tupleWriter;
+    this.providedSchema = providedSchema;
+    this.errorContext = errorContext;
+    this.conversions = new StandardConversions(providedSchema);
+  }
+
+  public ValueWriter makeWriter(String name, MinorType type, DataMode mode) {
+    ColumnMetadata providedCol = providedSchema == null ? null :
+        providedSchema.metadata(name);
+    if (providedCol == null) {
+      ColumnMetadata colSchema = MetadataUtils.newScalar(name, type, mode);
+      int index = tupleWriter.addColumn(colSchema);
+      return tupleWriter.scalar(index);
+    } else {
+      int index = tupleWriter.addColumn(providedCol);
+      return conversions.converter(tupleWriter.scalar(index), type);
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ExecutorFragmentContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ExecutorFragmentContext.java
index df782b1..030e2c4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ExecutorFragmentContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ExecutorFragmentContext.java
@@ -32,14 +32,12 @@ import java.util.Map;
 import java.util.Set;
 
 /**
- * This interface represents the context that is used by a Drillbit in classes like the
+ * The context that is used by a Drillbit in classes like the
  * {@link org.apache.drill.exec.work.fragment.FragmentExecutor}.
  */
 public interface ExecutorFragmentContext extends RootFragmentContext {
 
   /**
-   * Returns the root allocator for the Drillbit.
-   *
    * @return The root allocator for the Drillbit.
    */
   BufferAllocator getRootAllocator();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
index a1f25c1..adfd766 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
@@ -51,7 +51,7 @@ public class ImplCreator {
 
   private final LinkedList<CloseableRecordBatch> operators = Lists.newLinkedList();
 
-  private ImplCreator() {}
+  private ImplCreator() { }
 
   private List<CloseableRecordBatch> getOperators() {
     return operators;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OperatorCreatorRegistry.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OperatorCreatorRegistry.java
index 9150f91..09cef9a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OperatorCreatorRegistry.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OperatorCreatorRegistry.java
@@ -26,9 +26,11 @@ import java.util.Set;
 
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.scanner.persistence.ScanResult;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class OperatorCreatorRegistry {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(OperatorCreatorRegistry.class);
+  private static final Logger logger = LoggerFactory.getLogger(OperatorCreatorRegistry.class);
 
   private volatile Map<Class<?>, Constructor<?>> constructorRegistry = new HashMap<Class<?>, Constructor<?>>();
   private volatile Map<Class<?>, Object> instanceRegistry = new HashMap<Class<?>, Object>();
@@ -97,5 +99,4 @@ public class OperatorCreatorRegistry {
       }
     }
   }
-
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataHandlerBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataHandlerBatch.java
index 6c7d4df..6280364 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataHandlerBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataHandlerBatch.java
@@ -28,7 +28,7 @@ import org.apache.drill.exec.physical.config.MetadataHandlerPOP;
 import org.apache.drill.exec.metastore.analyze.AnalyzeColumnUtils;
 import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
 import org.apache.drill.exec.physical.resultSet.RowSetLoader;
-import org.apache.drill.exec.physical.resultSet.impl.OptionBuilder;
+import org.apache.drill.exec.physical.resultSet.impl.ResultSetOptionBuilder;
 import org.apache.drill.exec.physical.resultSet.impl.ResultSetLoaderImpl;
 import org.apache.drill.exec.physical.rowSet.DirectRowSet;
 import org.apache.drill.exec.physical.rowSet.RowSetReader;
@@ -291,8 +291,8 @@ public class MetadataHandlerBatch extends AbstractSingleRecordBatch<MetadataHand
         .addNullable(columnNamesOptions.lastModifiedTime(), MinorType.VARCHAR)
         .add(MetastoreAnalyzeConstants.METADATA_TYPE, MinorType.VARCHAR);
 
-    ResultSetLoaderImpl.ResultSetOptions options = new OptionBuilder()
-        .setSchema(schemaBuilder.buildSchema())
+    ResultSetLoaderImpl.ResultSetOptions options = new ResultSetOptionBuilder()
+        .readerSchema(schemaBuilder.buildSchema())
         .build();
 
     return new ResultSetLoaderImpl(container.getAllocator(), options);
@@ -390,8 +390,8 @@ public class MetadataHandlerBatch extends AbstractSingleRecordBatch<MetadataHand
       }
     }
 
-    ResultSetLoaderImpl.ResultSetOptions options = new OptionBuilder()
-        .setSchema(schemaBuilder.buildSchema())
+    ResultSetLoaderImpl.ResultSetOptions options = new ResultSetOptionBuilder()
+        .readerSchema(schemaBuilder.buildSchema())
         .build();
 
     return new ResultSetLoaderImpl(container.getAllocator(), options);
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 4a25426..778ef8e 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
@@ -68,9 +68,7 @@ import org.slf4j.LoggerFactory;
  * columns and must provide a message that helps the user identify
  * the likely original problem.
  */
-
 public class ColumnsArrayParser implements ScanProjectionParser {
-
   private static final Logger logger = LoggerFactory.getLogger(ColumnsArrayParser.class);
 
   // Config
@@ -79,7 +77,6 @@ public class ColumnsArrayParser implements ScanProjectionParser {
    * True if the project list must include either the columns[] array
    * or the wildcard.
    */
-
   private final boolean requireColumnsArray;
 
   /**
@@ -87,7 +84,6 @@ public class ColumnsArrayParser implements ScanProjectionParser {
    * the columns[] array. Handy if the plugin provides special columns such
    * as the log regex plugin.
    */
-
   private final boolean allowOtherCols;
 
   // Internals
@@ -174,7 +170,6 @@ public class ColumnsArrayParser implements ScanProjectionParser {
 
     // Special `columns` array column. Allow multiple, but
     // project only one.
-
     if (columnsArrayCol == null) {
       columnsArrayCol = new UnresolvedColumnsArrayColumn(inCol);
       builder.addTableColumn(columnsArrayCol);
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/AbstractConvertFromString.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/AbstractConvertFromString.java
similarity index 70%
rename from exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/AbstractConvertFromString.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/AbstractConvertFromString.java
index b3cd3f7..f31ee69 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/AbstractConvertFromString.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/AbstractConvertFromString.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.vector.accessor.convert;
+package org.apache.drill.exec.physical.impl.scan.convert;
 
 import java.util.Map;
 import java.util.function.Function;
@@ -24,42 +24,15 @@ import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.apache.drill.exec.vector.accessor.ScalarWriter;
 import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Abstract class for string-to-something conversions. Handles the
  * multiple ways that strings can be set.
  */
-public abstract class AbstractConvertFromString extends AbstractWriteConverter {
-
-  /**
-   * Property to control how the conversion handles blanks. Blanks are
-   * zero-length text fields (after trimming whitespace.)
-   */
-  public static final String BLANK_ACTION_PROP = "blank-as";
-
-  /**
-   * Convert blanks to null values (if the column is nullable), or
-   * fill with the default value (non-nullable.)
-   */
-  public static final String BLANK_AS_NULL = "null";
-
-  /**
-   * Convert blanks for numeric fields to 0. For non-numeric
-   * fields, convert to null (for nullable) or the default value
-   * (for non-nullable). Works best if non-numeric fields are declared
-   * as nullable.
-   */
-  public static final String BLANK_AS_ZERO = "0";
-
-  /**
-   * Skip blank values. This will result in the column being set to null for
-   * nullable columns, and to the default value (else 0 for numeric columns)
-   * for non-nullable columns.
-   */
-  public static final String BLANK_AS_SKIP = "skip";
-
-  protected static final org.slf4j.Logger logger =
-      org.slf4j.LoggerFactory.getLogger(AbstractConvertFromString.class);
+public abstract class AbstractConvertFromString extends DirectConverter {
+  protected static final Logger logger = LoggerFactory.getLogger(AbstractConvertFromString.class);
 
   protected final Function<String,String> prepare;
 
@@ -122,54 +95,42 @@ public abstract class AbstractConvertFromString extends AbstractWriteConverter {
    * @param properties optional framework-specific properties
    * @return a function to call to prepare each string value for conversion
    */
-
   private Function<String,String> buildPrepare(ColumnMetadata schema,
       Map<String, String> properties) {
 
-    String blankProp = schema.property(ColumnMetadata.BLANK_AS_PROP);
-    if (blankProp == null && properties != null) {
-      blankProp = properties.get(BLANK_ACTION_PROP);
-    }
-
+    String blankProp = properties.get(ColumnMetadata.BLANK_AS_PROP);
     if (blankProp != null) {
       switch (blankProp.toLowerCase()) {
-      case BLANK_AS_NULL:
-        return skipBlankFn(schema);
-      case BLANK_AS_ZERO:
-        return blankAsZeroFn(schema);
-     case BLANK_AS_SKIP:
-        if (schema.isNullable()) {
-          return blankToNullFn();
-        } else {
-          return blankAsZeroFn(schema);
-        }
-      default:
-        // Silently ignore invalid values
-        logger.warn("Invalid conversion option '{}', skipping", blankProp);
-        break;
+        case ColumnMetadata.BLANK_AS_ZERO:
+          if (!Types.isNumericType(schema.type())) {
+            return skipBlankFn(schema);
+          } else if (schema.isNullable()) {
+            return nullableBlankToZeroFn();
+          } else {
+            return blankToZeroFn();
+          }
+        case ColumnMetadata.BLANK_AS_NULL:
+          if (schema.isNullable()) {
+            return blankToNullFn();
+          } else {
+            return skipBlankFn(schema);
+          }
+        default:
+          // Silently ignore invalid values
+          logger.warn("Invalid conversion option '{}', skipping", blankProp);
+          break;
       }
     }
 
-    // Else, if the mode, then if the string is null, set the
+    // Else, if nullable, then if the string is null, set the
     // column to null, else trim the string.
-
     if (schema.isNullable()) {
       return nullableStrFn();
-    }
-
-    // Otherwise, trim the string, but have the converter skip this row if the
-    // string is blank. The column loader will then fill in the default value.
-
-    return skipBlanksFn();
-  }
-
-  private Function<String, String> blankAsZeroFn(ColumnMetadata schema) {
-    if (! Types.isNumericType(schema.type())) {
-      return skipBlankFn(schema);
-    } else if (schema.isNullable()) {
-      return nullableBlankToZeroFn();
     } else {
-      return blankToZeroFn();
+
+      // Otherwise, trim the string, but have the converter skip this row if the
+      // string is blank. The column loader will then fill in the default value.
+      return skipBlanksFn();
     }
   }
 
@@ -182,7 +143,13 @@ public abstract class AbstractConvertFromString extends AbstractWriteConverter {
   }
 
   private static Function<String, String> skipBlanksFn() {
-     return (String s) -> s == null ? s : s.trim();
+    return (String s) -> {
+      if (s == null) {
+        return null;
+      }
+      s = s.trim();
+      return s.isEmpty() ? null : s;
+    };
   }
 
   private Function<String, String> nullableStrFn() {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ColumnConverter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ColumnConverter.java
new file mode 100644
index 0000000..7d483b6
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ColumnConverter.java
@@ -0,0 +1,46 @@
+/*
+ * 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.convert;
+
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.UnsupportedConversionError;
+
+/**
+ * Base class for any kind of column converter. Extend this class if
+ * your columns work with reader-specific types, such as having to
+ * call a type-specific method on some row object. In this case,
+ * all your subclasses would implement the same method, say
+ * {@code void convertValue(MyRow row)}. Subclases might know their
+ * column indexes within the row object, extract the value using
+ * type-specific methods, then pass the value onto the scalar writer.
+ */
+public class ColumnConverter {
+  protected final ScalarWriter baseWriter;
+
+  public ColumnConverter(ScalarWriter colWriter) {
+    this.baseWriter = colWriter;
+  }
+
+  public ScalarWriter writer() { return baseWriter; }
+  public ColumnMetadata schema() { return baseWriter.schema(); }
+
+  protected UnsupportedConversionError conversionError(String javaType) {
+    return UnsupportedConversionError.writeError(schema(), javaType);
+  }
+}
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertBooleanToString.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertBooleanToString.java
similarity index 90%
rename from exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertBooleanToString.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertBooleanToString.java
index 159d81d..1be6d9cf 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertBooleanToString.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertBooleanToString.java
@@ -15,11 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.vector.accessor.convert;
+package org.apache.drill.exec.physical.impl.scan.convert;
 
 import org.apache.drill.exec.vector.accessor.ScalarWriter;
 
-public class ConvertBooleanToString extends AbstractWriteConverter {
+public class ConvertBooleanToString extends DirectConverter {
 
   public ConvertBooleanToString(ScalarWriter baseWriter) {
     super(baseWriter);
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertDateToString.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertDateToString.java
similarity index 94%
rename from exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertDateToString.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertDateToString.java
index 86f9ca0..136d0e7 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertDateToString.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertDateToString.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.vector.accessor.convert;
+package org.apache.drill.exec.physical.impl.scan.convert;
 
 import org.apache.drill.exec.vector.accessor.InvalidConversionError;
 import org.apache.drill.exec.vector.accessor.ScalarWriter;
@@ -24,7 +24,7 @@ import org.joda.time.format.DateTimeFormat;
 import org.joda.time.format.DateTimeFormatter;
 import org.joda.time.format.ISODateTimeFormat;
 
-public class ConvertDateToString extends AbstractWriteConverter {
+public class ConvertDateToString extends DirectConverter {
 
   private final DateTimeFormatter dateTimeFormatter;
 
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertDecimalToString.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertDecimalToString.java
similarity index 91%
rename from exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertDecimalToString.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertDecimalToString.java
index 8ada687..ce728d4 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertDecimalToString.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertDecimalToString.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.vector.accessor.convert;
+package org.apache.drill.exec.physical.impl.scan.convert;
 
 import java.math.BigDecimal;
 
@@ -23,7 +23,7 @@ import org.apache.drill.exec.vector.accessor.ScalarWriter;
 
 // TODO: provide some kind of format
 
-public class ConvertDecimalToString extends AbstractWriteConverter {
+public class ConvertDecimalToString extends DirectConverter {
 
   public ConvertDecimalToString(ScalarWriter baseWriter) {
     super(baseWriter);
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertDoubleToString.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertDoubleToString.java
similarity index 91%
rename from exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertDoubleToString.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertDoubleToString.java
index f0b216b..21dc779 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertDoubleToString.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertDoubleToString.java
@@ -15,13 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.vector.accessor.convert;
+package org.apache.drill.exec.physical.impl.scan.convert;
 
 import org.apache.drill.exec.vector.accessor.ScalarWriter;
 
 // TODO: provide some kind of format
 
-public class ConvertDoubleToString extends AbstractWriteConverter {
+public class ConvertDoubleToString extends DirectConverter {
 
   public ConvertDoubleToString(ScalarWriter baseWriter) {
     super(baseWriter);
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertIntToString.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertIntToString.java
similarity index 91%
copy from exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertIntToString.java
copy to exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertIntToString.java
index b869ea5..2e6c697 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertIntToString.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertIntToString.java
@@ -15,13 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.vector.accessor.convert;
+package org.apache.drill.exec.physical.impl.scan.convert;
 
 import org.apache.drill.exec.vector.accessor.ScalarWriter;
 
 // TODO: Provide some kind of format
 
-public class ConvertIntToString extends AbstractWriteConverter {
+public class ConvertIntToString extends DirectConverter {
 
   public ConvertIntToString(ScalarWriter baseWriter) {
     super(baseWriter);
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertIntervalToString.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertIntervalToString.java
similarity index 91%
rename from exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertIntervalToString.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertIntervalToString.java
index 4b1e58e..2999a14 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertIntervalToString.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertIntervalToString.java
@@ -15,14 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.vector.accessor.convert;
+package org.apache.drill.exec.physical.impl.scan.convert;
 
 import org.apache.drill.exec.vector.accessor.ScalarWriter;
 import org.joda.time.Period;
 
 // TODO: Provide some kind of format
 
-public class ConvertIntervalToString extends AbstractWriteConverter {
+public class ConvertIntervalToString extends DirectConverter {
 
   public ConvertIntervalToString(ScalarWriter baseWriter) {
     super(baseWriter);
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertLongToString.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertLongToString.java
similarity index 91%
rename from exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertLongToString.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertLongToString.java
index 5497814..e869c69 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertLongToString.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertLongToString.java
@@ -15,13 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.vector.accessor.convert;
+package org.apache.drill.exec.physical.impl.scan.convert;
 
 import org.apache.drill.exec.vector.accessor.ScalarWriter;
 
 // TODO: Provide some kind of format
 
-public class ConvertLongToString extends AbstractWriteConverter {
+public class ConvertLongToString extends DirectConverter {
 
   public ConvertLongToString(ScalarWriter baseWriter) {
     super(baseWriter);
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertStringToBoolean.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertStringToBoolean.java
similarity index 96%
rename from exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertStringToBoolean.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertStringToBoolean.java
index d7d4937..e77d0f1 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertStringToBoolean.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertStringToBoolean.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.vector.accessor.convert;
+package org.apache.drill.exec.physical.impl.scan.convert;
 
 import java.util.Map;
 
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertStringToDate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertStringToDate.java
similarity index 97%
rename from exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertStringToDate.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertStringToDate.java
index a3b0ba8..68c3c19 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertStringToDate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertStringToDate.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.vector.accessor.convert;
+package org.apache.drill.exec.physical.impl.scan.convert;
 
 import java.util.Map;
 
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertStringToDecimal.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertStringToDecimal.java
similarity index 96%
rename from exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertStringToDecimal.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertStringToDecimal.java
index 6c4944c..90e8c62 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertStringToDecimal.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertStringToDecimal.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.vector.accessor.convert;
+package org.apache.drill.exec.physical.impl.scan.convert;
 
 import java.math.BigDecimal;
 import java.util.Map;
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertStringToDouble.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertStringToDouble.java
similarity index 96%
rename from exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertStringToDouble.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertStringToDouble.java
index 9023b04..2fbceb6 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertStringToDouble.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertStringToDouble.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.vector.accessor.convert;
+package org.apache.drill.exec.physical.impl.scan.convert;
 
 import java.util.Map;
 
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertStringToInt.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertStringToInt.java
similarity index 96%
rename from exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertStringToInt.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertStringToInt.java
index f5f7531..5b9857e 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertStringToInt.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertStringToInt.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.vector.accessor.convert;
+package org.apache.drill.exec.physical.impl.scan.convert;
 
 import java.util.Map;
 
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertStringToInterval.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertStringToInterval.java
similarity index 96%
rename from exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertStringToInterval.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertStringToInterval.java
index ae61b9a..4dc0b20 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertStringToInterval.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertStringToInterval.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.vector.accessor.convert;
+package org.apache.drill.exec.physical.impl.scan.convert;
 
 import java.time.format.DateTimeParseException;
 import java.util.Map;
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertStringToLong.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertStringToLong.java
similarity index 96%
rename from exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertStringToLong.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertStringToLong.java
index a2bd704..051e4d7 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertStringToLong.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertStringToLong.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.vector.accessor.convert;
+package org.apache.drill.exec.physical.impl.scan.convert;
 
 import java.util.Map;
 
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertStringToTime.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertStringToTime.java
similarity index 97%
rename from exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertStringToTime.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertStringToTime.java
index 44c6412..945c395 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertStringToTime.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertStringToTime.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.vector.accessor.convert;
+package org.apache.drill.exec.physical.impl.scan.convert;
 
 import java.util.Map;
 
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertStringToTimeStamp.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertStringToTimeStamp.java
similarity index 97%
rename from exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertStringToTimeStamp.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertStringToTimeStamp.java
index 21c517e..d73f3ca 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertStringToTimeStamp.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertStringToTimeStamp.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.vector.accessor.convert;
+package org.apache.drill.exec.physical.impl.scan.convert;
 
 import java.util.Map;
 
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertTimeStampToString.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertTimeStampToString.java
similarity index 93%
rename from exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertTimeStampToString.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertTimeStampToString.java
index 89e7c6f..b17e0ec 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertTimeStampToString.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertTimeStampToString.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.vector.accessor.convert;
+package org.apache.drill.exec.physical.impl.scan.convert;
 
 import org.apache.drill.exec.vector.accessor.InvalidConversionError;
 import org.apache.drill.exec.vector.accessor.ScalarWriter;
@@ -24,7 +24,7 @@ import org.joda.time.format.DateTimeFormat;
 import org.joda.time.format.DateTimeFormatter;
 import org.joda.time.format.ISODateTimeFormat;
 
-public class ConvertTimeStampToString extends AbstractWriteConverter {
+public class ConvertTimeStampToString extends DirectConverter {
 
   private final DateTimeFormatter dateTimeFormatter;
 
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertTimeToString.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertTimeToString.java
similarity index 94%
rename from exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertTimeToString.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertTimeToString.java
index 9b6d1f7..d9222fa 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertTimeToString.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/ConvertTimeToString.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.vector.accessor.convert;
+package org.apache.drill.exec.physical.impl.scan.convert;
 
 import org.apache.drill.exec.vector.accessor.InvalidConversionError;
 import org.apache.drill.exec.vector.accessor.ScalarWriter;
@@ -24,7 +24,7 @@ import org.joda.time.format.DateTimeFormat;
 import org.joda.time.format.DateTimeFormatter;
 import org.joda.time.format.ISODateTimeFormat;
 
-public class ConvertTimeToString extends AbstractWriteConverter {
+public class ConvertTimeToString extends DirectConverter {
 
   private final DateTimeFormatter dateTimeFormatter;
 
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/AbstractWriteConverter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/DirectConverter.java
similarity index 55%
rename from exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/AbstractWriteConverter.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/DirectConverter.java
index 3f92ab1..5222926 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/AbstractWriteConverter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/DirectConverter.java
@@ -15,69 +15,32 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.vector.accessor.convert;
+package org.apache.drill.exec.physical.impl.scan.convert;
 
 import java.math.BigDecimal;
 
-import org.apache.drill.exec.record.metadata.ColumnMetadata;
-import org.apache.drill.exec.vector.accessor.ColumnReader;
-import org.apache.drill.exec.vector.accessor.ObjectType;
 import org.apache.drill.exec.vector.accessor.ScalarWriter;
-import org.apache.drill.exec.vector.accessor.ValueType;
-import org.apache.drill.exec.vector.accessor.writer.AbstractScalarWriter;
+import org.apache.drill.exec.vector.accessor.ValueWriter;
 import org.joda.time.Instant;
 import org.joda.time.LocalDate;
 import org.joda.time.LocalTime;
 import org.joda.time.Period;
 
 /**
- * Base class for type converting scalar column writers. All methods
- * pass through to the base writer. Override selected "set" methods to
- * perform the type conversion, such as overriding "setString" to convert
- * from a string representation of a value to the actual format.
+ * Base class for Java type-based conversion. Use this base class if your
+ * reader works with Java types, but you need to convert from one Java
+ * type to another (such as decoding a byte array, converting a string
+ * to a number, etc.)
  * <p>
- * The {@link #setObject(Object)} method works here: the object is passed
- * to this class's set methods, allowing, say, setting a string object
- * for an int column in the case above.
+ * Instances of this class can be freely mixed with "plain"
+ * {@code ScalarWriter} instances to avoid unnecessary calls when there
+ * is a direct mapping from reader Java type to Drill type. You only need
+ * insert converters for those columns where some conversion step is needed.
  */
+public class DirectConverter extends ColumnConverter implements ValueWriter {
 
-public abstract class AbstractWriteConverter extends AbstractScalarWriter {
-
-  protected final ScalarWriter baseWriter;
-
-  public AbstractWriteConverter(ScalarWriter baseWriter) {
-    this.baseWriter = baseWriter;
-  }
-
-  @Override
-  public ValueType valueType() {
-    return baseWriter.valueType();
-  }
-
-  @Override
-  public ObjectType type() {
-    return baseWriter.type();
-  }
-
-  @Override
-  public boolean isProjected() {
-    return baseWriter.isProjected();
-  }
-
-  @Override
-  public boolean nullable() {
-    return baseWriter.nullable();
-  }
-
-  @Override
-  public ColumnMetadata schema() {
-    return baseWriter.schema();
-  }
-
-  @Override
-  public void setDefaultValue(Object value) {
-    throw new IllegalStateException(
-        "Cannot set a default value through a shim; types conflict: " + value);
+  public DirectConverter(ScalarWriter colWriter) {
+    super(colWriter);
   }
 
   @Override
@@ -146,7 +109,7 @@ public abstract class AbstractWriteConverter extends AbstractScalarWriter {
   }
 
   @Override
-  public final void copy(ColumnReader from) {
-    throw new UnsupportedOperationException("Cannot copy values through a type converter");
+  public void setValue(Object value) {
+    baseWriter.setValue(value);
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/StandardConversions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/StandardConversions.java
new file mode 100644
index 0000000..7abaac5
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/StandardConversions.java
@@ -0,0 +1,478 @@
+/*
+ * 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.convert;
+
+import java.lang.reflect.Constructor;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.record.metadata.TupleSchema;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.ValueWriter;
+
+/**
+ * Factory for standard conversions as outlined in the package header.
+ * Can be used in two ways:
+ * <ul>
+ * <li>As an object with a default set of properties that apply to
+ * all columns.</li>
+ * <li>As a set of static functions which operate on each column
+ * individually.</li>
+ * </ul>
+ * The object form is more typical: it allows the provided schema to
+ * set general blank-handling behavior at the tuple level.
+ * <p>
+ * The class provides two kinds of information:
+ * <p>
+ * <ul>
+ * <li>A description of the conversion: whether the conversion is supported,
+ * whether it is lossy, and the class that can do the conversion.</li>
+ * <li>As a concrete conversion factory which provides the conversion class
+ * if needed, the original column writer if no conversion is needed, or
+ * {@code null} if the conversion is not supported.</li>
+ * </ul>
+ * <p>
+ * This class is not suitable if a reader requires a source-specific
+ * column converter or conversion rules. In that case, create a source-specific
+ * conversion factory.
+ */
+public class StandardConversions {
+
+  /**
+   * Indicates the type of conversion needed.
+   */
+  public enum ConversionType {
+
+    /**
+     * No conversion needed. Readers generally don't provide converters
+     * in this case unless the meaning of a column must change, keeping
+     * the type, such as converting between units.
+     */
+    NONE,
+
+    /**
+     * Conversion is done by the column writers. Again, no converter
+     * is needed except for semantic reasons.
+     */
+    IMPLICIT,
+
+    /**
+     * Conversion is done by the column writers. No converter is needed.
+     * However, the value is subject to overflow as this is a narrowing
+     * operation. Depending on the implementation, the operation may
+     * either raise an error, or produce a value limited by the range
+     * of the target type.
+     */
+    IMPLICIT_UNSAFE,
+
+    /**
+     * Conversion is needed because there is no "natural",
+     * precision-preserving conversion. Conversions must be done on
+     * an ad-hoc basis.
+     */
+    EXPLICIT
+  }
+
+  /**
+   * Definition of a conversion including conversion type and the standard
+   * conversion class (if available.)
+   */
+  public static class ConversionDefn {
+
+    public final ConversionType type;
+    public final Class<? extends DirectConverter> conversionClass;
+
+    public ConversionDefn(ConversionType type) {
+      this.type = type;
+      conversionClass = null;
+    }
+
+    public ConversionDefn(Class<? extends DirectConverter> conversionClass) {
+      this.type = ConversionType.EXPLICIT;
+      this.conversionClass = conversionClass;
+    }
+  }
+
+  public static final ConversionDefn IMPLICIT =
+      new ConversionDefn(ConversionType.IMPLICIT);
+  public static final ConversionDefn IMPLICIT_UNSAFE =
+      new ConversionDefn(ConversionType.IMPLICIT_UNSAFE);
+  public static final ConversionDefn EXPLICIT =
+      new ConversionDefn(ConversionType.EXPLICIT);
+
+  private final Map<String, String> properties;
+
+  public StandardConversions() {
+    this.properties = null;
+  }
+
+  public StandardConversions(Map<String,String> properties) {
+    this.properties = properties;
+  }
+
+  public StandardConversions(String blankAsProp) {
+    if (blankAsProp == null) {
+      this.properties = null;
+    } else {
+      this.properties = new HashMap<>();
+      this.properties.put(ColumnMetadata.BLANK_AS_PROP, blankAsProp);
+    }
+  }
+
+  public StandardConversions(TupleMetadata providedSchema) {
+    if (providedSchema == null || !providedSchema.hasProperties()) {
+      this.properties = null;
+    } else {
+      this.properties = providedSchema.properties();
+    }
+  }
+
+  private Map<String,String> merge(Map<String,String> specificProps) {
+    if (properties == null) {
+      return specificProps;
+    } else if (specificProps == null) {
+      return properties;
+    }
+    Map<String,String> merged = new HashMap<>();
+    merged.putAll(properties);
+    merged.putAll(specificProps);
+    return merged;
+  }
+
+  public static DirectConverter newInstance(
+      Class<? extends DirectConverter> conversionClass, ScalarWriter baseWriter,
+      Map<String,String> properties) {
+
+    // Try the Converter(ScalerWriter writer, Map<String, String> props) constructor first.
+    // This first form is optional.
+    try {
+      final Constructor<? extends DirectConverter> ctor = conversionClass.getDeclaredConstructor(ScalarWriter.class, Map.class);
+      return ctor.newInstance(baseWriter, properties);
+    } catch (final ReflectiveOperationException e) {
+      // Ignore
+    }
+
+    // Then try the Converter(ScalarSriter writer) constructor.
+    return newInstance(conversionClass, baseWriter);
+  }
+
+  public static DirectConverter newInstance(
+      Class<? extends DirectConverter> conversionClass, ScalarWriter baseWriter) {
+    try {
+      final Constructor<? extends DirectConverter> ctor = conversionClass.getDeclaredConstructor(ScalarWriter.class);
+      return ctor.newInstance(baseWriter);
+    } catch (final ReflectiveOperationException e) {
+      throw new IllegalStateException(e);
+    }
+  }
+
+  /**
+   * Create converters for standard cases.
+   * <p>
+   * Does not support any of the "legacy" decimal types.
+   *
+   * @param inputDefn the column schema for the input column which the
+   * client code (e.g. reader) wants to produce
+   * @param outputDefn the column schema for the output vector to be produced
+   * by this operator
+   * @return a description of the conversion needed (if any), along with the
+   * standard conversion class, if available
+   */
+  public static ConversionDefn analyze(ColumnMetadata inputSchema, ColumnMetadata outputSchema) {
+    return analyze(inputSchema.type(), outputSchema);
+  }
+
+  public static ConversionDefn analyze(MinorType inputType, ColumnMetadata outputSchema) {
+    if (inputType == outputSchema.type()) {
+      return new ConversionDefn(ConversionType.NONE);
+    }
+
+    switch (inputType) {
+      case VARCHAR:
+        return new ConversionDefn(convertFromVarchar(outputSchema));
+      case BIT:
+        switch (outputSchema.type()) {
+        case TINYINT:
+        case SMALLINT:
+        case INT:
+          return IMPLICIT;
+        case VARCHAR:
+          return new ConversionDefn(ConvertBooleanToString.class);
+        default:
+          break;
+        }
+        break;
+      case TINYINT:
+        switch (outputSchema.type()) {
+        case SMALLINT:
+        case INT:
+        case BIGINT:
+        case FLOAT4:
+        case FLOAT8:
+        case VARDECIMAL:
+          return IMPLICIT;
+        case VARCHAR:
+          return new ConversionDefn(ConvertIntToString.class);
+        default:
+          break;
+        }
+        break;
+      case SMALLINT:
+        switch (outputSchema.type()) {
+        case TINYINT:
+          return IMPLICIT_UNSAFE;
+        case INT:
+        case BIGINT:
+        case FLOAT4:
+        case FLOAT8:
+        case VARDECIMAL:
+          return IMPLICIT;
+        case VARCHAR:
+          return new ConversionDefn(ConvertIntToString.class);
+       default:
+          break;
+        }
+        break;
+      case INT:
+        switch (outputSchema.type()) {
+        case TINYINT:
+        case SMALLINT:
+          return IMPLICIT_UNSAFE;
+        case BIGINT:
+        case FLOAT4:
+        case FLOAT8:
+        case VARDECIMAL:
+        case TIME:
+          return IMPLICIT;
+        case VARCHAR:
+          return new ConversionDefn(ConvertIntToString.class);
+        default:
+          break;
+        }
+        break;
+      case BIGINT:
+        switch (outputSchema.type()) {
+        case TINYINT:
+        case SMALLINT:
+        case INT:
+          return IMPLICIT_UNSAFE;
+        case FLOAT4:
+        case FLOAT8:
+        case VARDECIMAL:
+        case DATE:
+        case TIMESTAMP:
+          return IMPLICIT;
+        case VARCHAR:
+          return new ConversionDefn(ConvertLongToString.class);
+        default:
+          break;
+        }
+        break;
+      case FLOAT4:
+        switch (outputSchema.type()) {
+        case TINYINT:
+        case SMALLINT:
+        case INT:
+        case BIGINT:
+          return IMPLICIT_UNSAFE;
+        case FLOAT8:
+        case VARDECIMAL:
+          return IMPLICIT;
+        case VARCHAR:
+          return new ConversionDefn(ConvertDoubleToString.class);
+        default:
+          break;
+        }
+        break;
+      case FLOAT8:
+        switch (outputSchema.type()) {
+        case TINYINT:
+        case SMALLINT:
+        case INT:
+        case BIGINT:
+        case FLOAT4:
+          return IMPLICIT_UNSAFE;
+        case VARDECIMAL:
+          return IMPLICIT;
+        case VARCHAR:
+          return new ConversionDefn(ConvertDoubleToString.class);
+        default:
+          break;
+        }
+        break;
+      case DATE:
+        switch (outputSchema.type()) {
+        case BIGINT:
+          return IMPLICIT;
+        case VARCHAR:
+          return new ConversionDefn(ConvertDateToString.class);
+        default:
+          break;
+        }
+        break;
+      case TIME:
+        switch (outputSchema.type()) {
+        case INT:
+          return IMPLICIT;
+        case VARCHAR:
+          return new ConversionDefn(ConvertTimeToString.class);
+        default:
+          break;
+        }
+        break;
+      case TIMESTAMP:
+        switch (outputSchema.type()) {
+        case BIGINT:
+          return IMPLICIT;
+        case VARCHAR:
+          return new ConversionDefn(ConvertTimeStampToString.class);
+        default:
+          break;
+        }
+        break;
+      case INTERVAL:
+      case INTERVALYEAR:
+      case INTERVALDAY:
+        switch (outputSchema.type()) {
+        case VARCHAR:
+          return new ConversionDefn(ConvertIntervalToString.class);
+        default:
+          break;
+        }
+        break;
+      default:
+        break;
+    }
+    return EXPLICIT;
+  }
+
+  public static Class<? extends DirectConverter> convertFromVarchar(
+      ColumnMetadata outputDefn) {
+    switch (outputDefn.type()) {
+      case BIT:
+        return ConvertStringToBoolean.class;
+      case TINYINT:
+      case SMALLINT:
+      case INT:
+      case UINT1:
+      case UINT2:
+        return ConvertStringToInt.class;
+      case BIGINT:
+        return ConvertStringToLong.class;
+      case FLOAT4:
+      case FLOAT8:
+        return ConvertStringToDouble.class;
+      case DATE:
+        return ConvertStringToDate.class;
+      case TIME:
+        return ConvertStringToTime.class;
+      case TIMESTAMP:
+        return ConvertStringToTimeStamp.class;
+      case INTERVALYEAR:
+      case INTERVALDAY:
+      case INTERVAL:
+        return ConvertStringToInterval.class;
+      case VARDECIMAL:
+        return ConvertStringToDecimal.class;
+      default:
+        return null;
+    }
+  }
+
+  /**
+   * Create a direct column converter, if necessary, for the given input type
+   * and optional properties. The properties provide formats and other type
+   * conversion hints needed for some conversions.
+   *
+   * @param scalarWriter the output column writer
+   * @param inputType the type of the input data
+   * @param properties optional properties for some string-based conversions
+   * @return a column converter, if needed and available, the input writer if
+   * no conversion is needed, or null if there is no conversion available
+   */
+  public static ValueWriter converterFor(ScalarWriter scalarWriter, MinorType inputType, Map<String, String> properties) {
+    ConversionDefn defn = analyze(inputType, scalarWriter.schema());
+    switch (defn.type) {
+      case EXPLICIT:
+        if (defn.conversionClass != null) {
+          return newInstance(defn.conversionClass, scalarWriter, properties);
+        } else {
+          return null;
+        }
+      case IMPLICIT:
+      case IMPLICIT_UNSAFE:
+        return newInstance(defn.conversionClass, scalarWriter, properties);
+      default:
+        return scalarWriter;
+    }
+  }
+
+  public static ValueWriter converterFor(ScalarWriter scalarWriter, MinorType inputType) {
+    return converterFor(scalarWriter, inputType, null);
+  }
+
+  public static ValueWriter converterFor(ScalarWriter scalarWriter, ColumnMetadata inputSchema) {
+    return converterFor(scalarWriter, inputSchema.type(), inputSchema.properties());
+  }
+
+  public ValueWriter converter(ScalarWriter scalarWriter, ColumnMetadata inputSchema) {
+    return converterFor(scalarWriter, inputSchema.type(),
+        merge(inputSchema.properties()));
+  }
+
+  public ValueWriter converter(ScalarWriter scalarWriter, MinorType inputType) {
+    return converterFor(scalarWriter, inputType, properties);
+  }
+
+  /**
+   * Given a desired provided schema and an actual reader schema, create a merged
+   * schema that contains the provided column where available, but the reader
+   * column otherwise. Copies provided properties to the output schema.
+   * <p>
+   * The result is the schema to use when creating column writers: it reflects
+   * the type of the target vector. The reader is responsible for converting from
+   * the (possibly different) reader column type to the provided column type.
+   * <p>
+   * Note: the provided schema should only contain types that the reader is prepared
+   * to offer: there is no requirement that the reader support every possible conversion,
+   * only those that make sense for that one reader.
+   *
+   * @param providedSchema the provided schema from {@code CREATE SCHEMA}
+   * @param readerSchema the set of column types that the reader can provide
+   * "natively"
+   * @return a merged schema to use when creating the {@code ResultSetLoader}
+   */
+  public static TupleMetadata mergeSchemas(TupleMetadata providedSchema,
+      TupleMetadata readerSchema) {
+    if (providedSchema == null) {
+      return readerSchema;
+    }
+    final TupleMetadata tableSchema = new TupleSchema();
+    for (ColumnMetadata readerCol : readerSchema) {
+      final ColumnMetadata providedCol = providedSchema.metadata(readerCol.name());
+      tableSchema.addColumn(providedCol == null ? readerCol : providedCol);
+    }
+    if (providedSchema.hasProperties()) {
+      tableSchema.properties().putAll(providedSchema.properties());
+    }
+    return tableSchema;
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/package-info.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/package-info.java
new file mode 100644
index 0000000..150feb4
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/convert/package-info.java
@@ -0,0 +1,78 @@
+/*
+ * 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.
+ */
+
+/**
+ * Standard type conversion tools for the case in which the input
+ * types are the standard Java types already supported by the
+ * {@link ValuesWriter} interface. The classes here wrap a
+ * {@link ScalarWriter}: either a simple column, or the scalar portion of
+ * an array column.
+ * <p>
+ * The converters sit outside of the column writer hierarchy. In the
+ * most general case, a batch reader uses a "column adapter" to convert
+ * from some source-specific input format such as a JDBC {@code ResultSet}.
+ * In this case, the reader maintains a "row format": the set of column
+ * adapters that make up the row.
+ * <p>
+ * The classes here handle a special case: when the reader just needs
+ * the standard Java types. These classes allow a batch reader to intermix
+ * "plain" column writers and column conversions in a "row format".
+ * This in turn, ensures
+ * that there is no extra function call overhead when no conversion
+ * is needed.
+ * <p>
+ * Provides a mapping from input to output
+ * (vector) type. Handles implicit conversions (those done by the column
+ * writer itself) and explicit conversions for the Java string notation
+ * for various types. Readers must provide custom conversions or specialized
+ * formats.
+ * <p>
+ * Type-narrowing operations are supported by the column writers using
+ * "Java semantics". Long-to-int overflow is caught, double-to-long conversion
+ * sets the maximum or minimum long values, Double-to-int will overflow (on
+ * the int conversion.) Messy, but the goal is not to handle invalid data,
+ * rather it is to provide convenience for valid data.
+ * <p>
+ * The semantics of invalid conversions can be refined (set to null or
+ * throw an exception) without affecting the behavior of queries with
+ * valid data.
+ * <p>
+ * The provided conversions all handle the normal cases. Exceptional
+ * case (overflow, ambiguous formats) are handled according to Java
+ * rules.
+ * <p>
+ * The {@link StandardConversions} class defines the types of conversions
+ * needed:
+ * <ul>
+ * <li>None: Converting from a type to itself.</li>
+ * <li>Implicit: Conversion is done by the column writers, such as
+ * converting from an INT to a SMALLINT.</li>
+ * <li>Explicit: Requires a converter. If an unambiguous conversion is
+ * possible, that converter should occur here. If conversion is ambiguous,
+ * or a reader needs to support a special format, then the reader can add
+ * custom conversions for these cases.</li>
+ * </ul>
+ * <p>
+ * Would be good to validate each conversion against the corresponding CAST
+ * operation. In an ideal world, all conversions, normal and exceptional,
+ * will work the same as either a CAST (where the operations is handled by
+ * the Project operator via code generation) and the standard conversions.
+ */
+package org.apache.drill.exec.physical.impl.scan.convert;
+
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
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 02d4da9..cb51c76 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
@@ -39,7 +39,7 @@ public class FileMetadataColumnsParser implements ScanProjectionParser {
 
   // Internal
 
-  private final FileMetadataManager metadataManager;
+  private final ImplicitColumnManager metadataManager;
   private final Pattern partitionPattern;
   private ScanLevelProjection builder;
   private final Set<Integer> referencedPartitions = new HashSet<>();
@@ -48,7 +48,7 @@ public class FileMetadataColumnsParser implements ScanProjectionParser {
 
   private boolean hasImplicitCols;
 
-  public FileMetadataColumnsParser(FileMetadataManager metadataManager) {
+  public FileMetadataColumnsParser(ImplicitColumnManager metadataManager) {
     this.metadataManager = metadataManager;
     partitionPattern = Pattern.compile(metadataManager.partitionDesignator + "(\\d+)", Pattern.CASE_INSENSITIVE);
   }
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 8c6cd22..da2bf48 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
@@ -27,7 +27,7 @@ import org.apache.drill.common.exceptions.CustomErrorContext;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.exceptions.UserException.Builder;
 import org.apache.drill.exec.physical.impl.scan.ScanOperatorEvents;
-import org.apache.drill.exec.physical.impl.scan.file.FileMetadataManager.FileMetadataOptions;
+import org.apache.drill.exec.physical.impl.scan.file.ImplicitColumnManager.ImplicitColumnOptions;
 import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
 import org.apache.drill.exec.physical.impl.scan.framework.ManagedScanFramework;
 import org.apache.drill.exec.physical.impl.scan.framework.SchemaNegotiator;
@@ -38,6 +38,8 @@ import org.apache.drill.exec.store.dfs.easy.FileWork;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * The file scan framework adds into the scan framework support for implicit
@@ -60,11 +62,8 @@ import org.apache.hadoop.mapred.FileSplit;
  * <p>
  * @See {AbstractScanFramework} for details.
  */
-
 public class FileScanFramework extends ManagedScanFramework {
-
-  private static final org.slf4j.Logger logger =
-      org.slf4j.LoggerFactory.getLogger(FileScanFramework.class);
+  private static final Logger logger = LoggerFactory.getLogger(FileScanFramework.class);
 
   /**
    * The file schema negotiator adds no behavior at present, but is
@@ -76,7 +75,6 @@ public class FileScanFramework extends ManagedScanFramework {
    * exists. Those are out of scope of this first round of changes which
    * focus on schema.
    */
-
   public interface FileSchemaNegotiator extends SchemaNegotiator {
 
     /**
@@ -98,7 +96,6 @@ public class FileScanFramework extends ManagedScanFramework {
    * file-specific features exist. This class shows, however, where we would
    * add such features.
    */
-
   public static class FileSchemaNegotiatorImpl extends SchemaNegotiatorImpl
       implements FileSchemaNegotiator {
 
@@ -141,13 +138,12 @@ public class FileScanFramework extends ManagedScanFramework {
   /**
    * Options for a file-based scan.
    */
-
   public static class FileScanBuilder extends ScanFrameworkBuilder {
     private List<? extends FileWork> files;
     private Configuration fsConf;
-    private FileMetadataOptions metadataOptions = new FileMetadataOptions();
+    private final ImplicitColumnOptions metadataOptions = new ImplicitColumnOptions();
 
-    public void setConfig(Configuration fsConf) {
+    public void setFileSystemConfig(Configuration fsConf) {
       this.fsConf = fsConf;
     }
 
@@ -155,7 +151,7 @@ public class FileScanFramework extends ManagedScanFramework {
       this.files = files;
     }
 
-    public FileMetadataOptions metadataOptions() { return metadataOptions; }
+    public ImplicitColumnOptions implicitColumnOptions() { return metadataOptions; }
 
     @Override
     public ScanOperatorEvents buildEvents() {
@@ -170,7 +166,6 @@ public class FileScanFramework extends ManagedScanFramework {
    * makes clear that the constructor for the reader should do nothing;
    * work should be done in the open() call.
    */
-
   public abstract static class FileReaderFactory implements ReaderFactory {
 
     private FileScanFramework fileFramework;
@@ -195,9 +190,9 @@ public class FileScanFramework extends ManagedScanFramework {
     public abstract ManagedReader<? extends FileSchemaNegotiator> newReader();
   }
 
-  private FileMetadataManager metadataManager;
+  private ImplicitColumnManager metadataManager;
   private DrillFileSystem dfs;
-  private List<FileSplit> spilts = new ArrayList<>();
+  private final List<FileSplit> spilts = new ArrayList<>();
   private Iterator<FileSplit> splitIter;
   private FileSplit currentSplit;
 
@@ -241,12 +236,11 @@ public class FileScanFramework extends ManagedScanFramework {
 
     // Create the metadata manager to handle file metadata columns
     // (so-called implicit columns and partition columns.)
-
-    options.metadataOptions().setFiles(paths);
-    metadataManager = new FileMetadataManager(
+    options.implicitColumnOptions().setFiles(paths);
+    metadataManager = new ImplicitColumnManager(
         context.getFragmentContext().getOptions(),
-        options.metadataOptions());
-    builder.withMetadata(metadataManager);
+        options.implicitColumnOptions());
+    builder.withImplicitColumns(metadataManager);
   }
 
   protected FileSplit nextSplit() {
@@ -258,7 +252,6 @@ public class FileScanFramework extends ManagedScanFramework {
 
     // Tell the metadata manager about the current file so it can
     // populate the metadata columns, if requested.
-
     metadataManager.startFile(currentSplit.getPath());
     return currentSplit;
   }
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/ImplicitColumnManager.java
similarity index 96%
rename from exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileMetadataManager.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/ImplicitColumnManager.java
index e859c46..92fb845 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/ImplicitColumnManager.java
@@ -60,7 +60,7 @@ import org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTes
  * <p>
  * This is the successor to {@link org.apache.drill.exec.store.ColumnExplorer}.
  */
-public class FileMetadataManager implements MetadataManager, ReaderProjectionResolver, VectorSource {
+public class ImplicitColumnManager implements MetadataManager, ReaderProjectionResolver, VectorSource {
 
   /**
    * Automatically compute partition depth from files. Use only
@@ -68,7 +68,7 @@ public class FileMetadataManager implements MetadataManager, ReaderProjectionRes
    */
   public static final int AUTO_PARTITION_DEPTH = -1;
 
-  public static class FileMetadataOptions {
+  public static class ImplicitColumnOptions {
 
     private Path rootDir;
     private int partitionCount = AUTO_PARTITION_DEPTH;
@@ -119,7 +119,7 @@ public class FileMetadataManager implements MetadataManager, ReaderProjectionRes
 
   // Input
 
-  private final FileMetadataOptions options;
+  private final ImplicitColumnOptions options;
   private FileMetadata currentFile;
 
   // Config
@@ -156,8 +156,8 @@ public class FileMetadataManager implements MetadataManager, ReaderProjectionRes
    * depth across all readers in this fragment
    */
 
-  public FileMetadataManager(OptionSet optionManager,
-      FileMetadataOptions config) {
+  public ImplicitColumnManager(OptionSet optionManager,
+      ImplicitColumnOptions config) {
     this.options = config;
 
     partitionDesignator = optionManager.getString(ExecConstants.FILESYSTEM_PARTITION_COLUMN_LABEL);
@@ -199,7 +199,7 @@ public class FileMetadataManager implements MetadataManager, ReaderProjectionRes
     }
   }
 
-  protected FileMetadataOptions options() { return options; }
+  protected ImplicitColumnOptions options() { return options; }
 
   private int computeMaxPartition(List<Path> files) {
     int maxLen = 0;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/PartitionColumn.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/PartitionColumn.java
index e1f9215..1ddf227 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/PartitionColumn.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/PartitionColumn.java
@@ -26,7 +26,6 @@ import org.apache.drill.exec.physical.impl.scan.project.VectorSource;
  * Represents a partition column (usually dir0, dir1, ...). This is an implicit
  * column that has a variable part: the partition index.
  */
-
 public class PartitionColumn extends MetadataColumn {
 
   protected final int partition;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/ManagedScanFramework.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/ManagedScanFramework.java
index 095372c..860912c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/ManagedScanFramework.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/ManagedScanFramework.java
@@ -23,6 +23,7 @@ import org.apache.drill.exec.physical.impl.scan.RowBatchReader;
 import org.apache.drill.exec.physical.impl.scan.ScanOperatorEvents;
 import org.apache.drill.exec.physical.impl.scan.project.ScanSchemaOrchestrator;
 import org.apache.drill.exec.physical.impl.scan.project.ScanSchemaOrchestrator.ScanOrchestratorBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -111,9 +112,7 @@ import org.slf4j.LoggerFactory;
  * <li>The reader then releases its resources.</li>
  * </ul>
  */
-
 public class ManagedScanFramework implements ScanOperatorEvents {
-
   static final Logger logger = LoggerFactory.getLogger(ManagedScanFramework.class);
 
   /**
@@ -129,7 +128,6 @@ public class ManagedScanFramework implements ScanOperatorEvents {
    * Also manages opening the reader using a scan-specific schema
    * negotiator.
    */
-
   public interface ReaderFactory {
     void bind(ManagedScanFramework framework);
     ManagedReader<? extends SchemaNegotiator> next();
@@ -185,6 +183,10 @@ public class ManagedScanFramework implements ScanOperatorEvents {
     return scanOrchestrator;
   }
 
+  public TupleMetadata outputSchema() {
+    return scanOrchestrator.providedSchema();
+  }
+
   public CustomErrorContext errorContext() { return builder.errorContext(); }
 
   protected void configure() { }
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 2be5ab6..7c9a753 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
@@ -25,28 +25,8 @@ import org.apache.drill.exec.record.metadata.TupleMetadata;
 
 /**
  * Negotiates the table schema with the scanner framework and provides
- * context information for the reader. In a typical scan, the physical
- * plan provides the project list: the set of columns that the query
- * expects. Readers provide a table schema: the set of columns actually
- * available. The scan framework combines the two lists to determine
- * the available table columns that must be read, along with any additional
- * to be added. Additional columns can be file metadata (if the storage
- * plugin requests them), or can be null columns added for projected
- * columns that don't actually exist in the table.
- * <p>
- * The reader provides the table schema in one of two ways:
- * <ul>
- * <li>If the reader is of "early schema" type, then the reader calls
- * {@link #setTableSchema(TupleMetadata)} to provide that schema.</li>
- * <li>If the reader is of "late schema" type, then the reader discovers
- * the schema as the data is read, calling the
- * {@link RowSetLoader#addColumn()} method to add each column as it is
- * discovered.
- * <p>
- * Either way, the project list from the physical plan determines which
- * table columns are materialized and which are not. Readers are provided
- * for all table columns for readers that must read sequentially, but
- * only the materialized columns are written to value vectors.
+ * context information for the reader. Scans use either a "dynamic" or
+ * a defined schema.
  * <p>
  * Regardless of the schema type, the result of building the schema is a
  * result set loader used to prepare batches for use in the query. The reader
@@ -54,8 +34,69 @@ import org.apache.drill.exec.record.metadata.TupleMetadata;
  * values. Or for efficiency, the reader can check the column metadata to
  * determine if a column is projected, and if not, then don't even read
  * the column from the input source.
+ *
+ * <h4>Defined Schema</h4>
+ *
+ * If defined, the execution plan provides the output schema (presumably
+ * computed from an accurate metadata source.) The reader must populate
+ * the proscribed rows, performing column type conversions as needed.
+ * The reader can determine if the schema is defined by calling
+ * {@link hasOutputSchema()}.
+ * <p>
+ * At present, the scan framework filters the "provided schema" against
+ * the project list so that this class presents only the actual output
+ * schema. Future versions may do the filtering in the planner, but
+ * the result for readers will be the same either way.
+ *
+ * <h4>Dynamic Schema</h4>
+ *
+ * A dynamic schema occurs when the plan does not specify a schema.
+ * Drill is unique in its support for "schema on read" in the sense
+ * that Drill does not know the schema until the reader defines it at
+ * scan time.
+ * <p>
+ * The reader and scan framework coordinate to form the output schema.
+ * The reader offers the columns it has available. The scan framework
+ * uses the projection list to decide which to accept. Either way the
+ * scan framework provides a column reader for the column (returning a
+ * do-nothing "dummy" reader if the column is unprojected.)
+ * <p>
+ * With a dynamic schema, readers offer a schema in one of two ways:
+ * <p>
+ * The reader provides the table schema in one of two ways: early schema
+ * or late schema. Either way, the project list from the physical plan
+ * determines which
+ * table columns are materialized and which are not. Readers are provided
+ * for all table columns for readers that must read sequentially, but
+ * only the materialized columns are written to value vectors.
+ *
+ * <h4>Early Dynamic Schema</h4>
+ *
+ * Some readers can determine the source schema at the start of a scan.
+ * For example, a CSV file has headers, a Parquet file has footers, both
+ * of which define a schema. This case is called "early schema." The
+ * reader fefines the schema by calling
+ * {@link #tableSchema(TupleMetadata)} to provide the known schema.
+ *
+ * <h4>Late Dynamic Schema</h4>
+ *
+ * Other readers don't know the input schema until the reader actually
+ * reads the data. For example, JSON typically has no schema, but does
+ * have sufficient structure (name/value pairs) to infer one.
+ * <p>
+ * The late schema reader calls {@link RowSetLoader#addColumn()} to
+ * add each column as it is discovered during the scan.
+ * <p>
+ * Note that, to avoid schema conflicts, a late schema reader
+ * <i><b>must</b></i> define the full set of columns in the first batch,
+ * and must stick to that schema for all subsequent batches. This allows
+ * the reader to look one batch ahead to learn the columns.
+ * <p>
+ * Drill, however, cannot predict the future. Without a defined schema,
+ * downstream operators cannot know which columns might appear later
+ * in the scan, with which types. Today this is a strong guideline.
+ * Future versions may enforce this rule.
  */
-
 public interface SchemaNegotiator {
 
   OperatorContext context();
@@ -64,20 +105,43 @@ public interface SchemaNegotiator {
    * Specify an advanced error context which allows the reader to
    * fill in custom context values.
    */
-
   void setErrorContext(CustomErrorContext context);
 
-  /*
-   * The name of the user running the query.
+  /**
+   * Name of the user running the query.
    */
-
   String userName();
 
   /**
+   * Report if the execution plan defines a provided schema. If so,
+   * the reader should use that schema, converting or ignoring columns
+   * as needed. A scan without a provided schema has a "dynamic" schema
+   * to be defined by the scan operator itself along with the column
+   * projection list.
+   *
+   * @return {@code true} if the execution plan defines the output
+   * schema, {@code false} if the schema should be computed dynamically
+   * from the source schema and column projections
+   */
+  boolean hasProvidedSchema();
+
+  /**
+   * Returns the provided schema, if defined. The provided schema is a
+   * description of the source schema viewed as a Drill schema.
+   *
+   * @return the output schema, if {@link #hasProvidedSchema()} returns
+   * {@code true}, {@code null} otherwise
+   */
+  TupleMetadata providedSchema();
+
+  /**
    * Specify the table schema if this is an early-schema reader. Need
    * not be called for a late-schema readers. The schema provided here,
    * if any, is a base schema: the reader is free to discover additional
    * columns during the read.
+   * <p>
+   * Should only be called if the schema is dynamic, that is, if
+   * {@link #hasProvidedSchema()} returns false.
    *
    * @param schema the table schema if known at open time
    * @param isComplete true if the schema is complete: if it can be used
@@ -85,8 +149,7 @@ public interface SchemaNegotiator {
    * false if the schema is partial: if the reader must read rows to
    * determine the full schema
    */
-
-  void setTableSchema(TupleMetadata schema, boolean isComplete);
+  void tableSchema(TupleMetadata schema, boolean isComplete);
 
   /**
    * Set the preferred batch size (which may be overridden by the
@@ -94,8 +157,7 @@ public interface SchemaNegotiator {
    *
    * @param maxRecordsPerBatch preferred number of record per batch
    */
-
-  void setBatchSize(int maxRecordsPerBatch);
+  void batchSize(int maxRecordsPerBatch);
 
   /**
    * Build the schema, plan the required projections and static
@@ -108,7 +170,6 @@ public interface SchemaNegotiator {
    * @return the loader for the table with columns arranged in table
    * schema order
    */
-
   ResultSetLoader build();
 
   /**
@@ -123,7 +184,6 @@ public interface SchemaNegotiator {
    * row count, false if at least one column is projected and so
    * data must be written using the loader
    */
-
   boolean isProjectionEmpty();
 
   /**
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 e1302218..8763dd4 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
@@ -48,7 +48,6 @@ import org.apache.drill.exec.vector.ValueVector;
  * filled in by the scan projector (assuming, of course, that "c"
  * is nullable or an array.)
  */
-
 public class SchemaNegotiatorImpl implements SchemaNegotiator {
 
   public interface NegotiatorListener {
@@ -58,12 +57,14 @@ public class SchemaNegotiatorImpl implements SchemaNegotiator {
   protected final ManagedScanFramework framework;
   private NegotiatorListener listener;
   protected CustomErrorContext context;
+  protected TupleMetadata providedSchema;
   protected TupleMetadata tableSchema;
   protected boolean isSchemaComplete;
   protected int batchSize = ValueVector.MAX_ROW_COUNT;
 
   public SchemaNegotiatorImpl(ManagedScanFramework framework) {
     this.framework = framework;
+    this.providedSchema = framework.outputSchema();
   }
 
   public void bind(NegotiatorListener listener) {
@@ -71,6 +72,23 @@ public class SchemaNegotiatorImpl implements SchemaNegotiator {
   }
 
   @Override
+  public boolean isProjectionEmpty() {
+    return framework.scanOrchestrator().isProjectNone();
+  }
+
+  @Override
+  public boolean hasProvidedSchema() {
+    // Does not count as an output schema if no columns
+    // (only properties) are provided.
+    return providedSchema != null && providedSchema.size() > 0;
+  }
+
+  @Override
+  public TupleMetadata providedSchema() {
+    return providedSchema;
+  }
+
+  @Override
   public OperatorContext context() {
     return framework.context();
   }
@@ -90,13 +108,15 @@ public class SchemaNegotiatorImpl implements SchemaNegotiator {
   }
 
   @Override
-  public void setTableSchema(TupleMetadata schema, boolean isComplete) {
+  public void tableSchema(TupleMetadata schema, boolean isComplete) {
     tableSchema = schema;
     isSchemaComplete = schema != null && isComplete;
   }
 
+  public boolean isSchemaComplete() { return tableSchema != null && isSchemaComplete; }
+
   @Override
-  public void setBatchSize(int maxRecordsPerBatch) {
+  public void batchSize(int maxRecordsPerBatch) {
     batchSize = maxRecordsPerBatch;
   }
 
@@ -114,19 +134,10 @@ public class SchemaNegotiatorImpl implements SchemaNegotiator {
    * schema information
    * @return the result set loader to be used by the reader
    */
-
   @Override
   public ResultSetLoader build() {
 
     // Build and return the result set loader to be used by the reader.
-
     return listener.build(this);
   }
-
-  @Override
-  public boolean isProjectionEmpty() {
-    return framework.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 1f91e2a..ec16699 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
@@ -23,6 +23,8 @@ import org.apache.drill.exec.physical.impl.scan.framework.SchemaNegotiatorImpl.N
 import org.apache.drill.exec.physical.impl.scan.project.ReaderSchemaOrchestrator;
 import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
 import org.apache.drill.exec.record.VectorContainer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Represents a layer of row batch reader that works with a
@@ -37,10 +39,8 @@ import org.apache.drill.exec.record.VectorContainer;
  * this class abstracts out the schema logic. This allows a variety
  * of solutions as needed for different readers.
  */
-
 public class ShimBatchReader implements RowBatchReader, NegotiatorListener {
-
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ShimBatchReader.class);
+  static final Logger logger = LoggerFactory.getLogger(ShimBatchReader.class);
 
   protected final ManagedScanFramework framework;
   protected final ManagedReader<? extends SchemaNegotiator> reader;
@@ -52,7 +52,6 @@ public class ShimBatchReader implements RowBatchReader, NegotiatorListener {
    * True once the reader reports EOF. This shim may keep going for another
    * batch to handle any look-ahead row on the last batch.
    */
-
   private boolean eof;
 
   public ShimBatchReader(ManagedScanFramework manager, ManagedReader<? extends SchemaNegotiator> reader) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/AbstractUnresolvedColumn.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/AbstractUnresolvedColumn.java
index 78bc388..05d1df2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/AbstractUnresolvedColumn.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/AbstractUnresolvedColumn.java
@@ -32,7 +32,6 @@ import org.apache.drill.exec.record.metadata.ColumnMetadata;
  * null column builder, etc.) to the output batch. Thus the columns
  * here are placeholders to be rewritten once more data is available.
  */
-
 public abstract class AbstractUnresolvedColumn implements ColumnProjection {
 
   /**
@@ -40,7 +39,6 @@ public abstract class AbstractUnresolvedColumn implements ColumnProjection {
    * reader (or filled in with nulls.) May be associated with
    * a provided schema column.
    */
-
   public static class UnresolvedColumn extends AbstractUnresolvedColumn {
 
     private final ColumnMetadata colDefn;
@@ -72,7 +70,6 @@ public abstract class AbstractUnresolvedColumn implements ColumnProjection {
    * maps. In some cases, multiple output columns map map the to the
    * same "input" (to the projection process) column.
    */
-
   protected final RequestedColumn inCol;
 
   public AbstractUnresolvedColumn(RequestedColumn inCol) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ReaderLevelProjection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ReaderLevelProjection.java
index 8b46d88..ff14269 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ReaderLevelProjection.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ReaderLevelProjection.java
@@ -58,7 +58,6 @@ import org.apache.drill.exec.record.metadata.TupleMetadata;
  * @see {@link ImplicitColumnExplorer}, the class from which this class
  * evolved
  */
-
 public class ReaderLevelProjection {
 
   /**
@@ -68,7 +67,6 @@ public class ReaderLevelProjection {
    * final projected columns. The metadata manager, for example, implements
    * this interface to map metadata columns.
    */
-
   public interface ReaderProjectionResolver {
     void startResolution();
     boolean resolveColumn(ColumnProjection col, ResolvedTuple tuple,
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
index 99356e1..907390e 100644
--- 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
@@ -20,9 +20,8 @@ package org.apache.drill.exec.physical.impl.scan.project;
 import org.apache.drill.common.exceptions.CustomErrorContext;
 import org.apache.drill.exec.physical.impl.scan.project.NullColumnBuilder.NullBuilderBuilder;
 import org.apache.drill.exec.physical.impl.scan.project.ResolvedTuple.ResolvedRow;
-import org.apache.drill.exec.physical.impl.scan.project.projSet.ProjectionSetBuilder;
 import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
-import org.apache.drill.exec.physical.resultSet.impl.OptionBuilder;
+import org.apache.drill.exec.physical.resultSet.impl.ResultSetOptionBuilder;
 import org.apache.drill.exec.physical.resultSet.impl.ResultSetLoaderImpl;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
@@ -67,11 +66,11 @@ public class ReaderSchemaOrchestrator implements VectorSource {
   }
 
   public ResultSetLoader makeTableLoader(CustomErrorContext errorContext, TupleMetadata readerSchema) {
-    OptionBuilder options = new OptionBuilder();
-    options.setRowCountLimit(Math.min(readerBatchSize, scanOrchestrator.options.scanBatchRecordLimit));
-    options.setVectorCache(scanOrchestrator.vectorCache);
-    options.setBatchSizeLimit(scanOrchestrator.options.scanBatchByteLimit);
-    options.setContext(errorContext);
+    ResultSetOptionBuilder options = new ResultSetOptionBuilder();
+    options.rowCountLimit(Math.min(readerBatchSize, scanOrchestrator.options.scanBatchRecordLimit));
+    options.vectorCache(scanOrchestrator.vectorCache);
+    options.batchSizeLimit(scanOrchestrator.options.scanBatchByteLimit);
+    options.errorContext(errorContext);
 
     // Set up a selection list if available and is a subset of
     // table columns. (Only needed for non-wildcard queries.)
@@ -79,11 +78,8 @@ public class ReaderSchemaOrchestrator implements VectorSource {
     // 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.
-
-    ProjectionSetBuilder projBuilder = scanOrchestrator.scanProj.projectionSet();
-    projBuilder.typeConverter(scanOrchestrator.options.typeConverter);
-    options.setProjection(projBuilder.build());
-    options.setSchema(readerSchema);
+    options.projectionFilter(scanOrchestrator.scanProj.readerProjection);
+    options.readerSchema(readerSchema);
 
     // Create the table loader
     tableLoader = new ResultSetLoaderImpl(scanOrchestrator.allocator, options.build());
@@ -150,26 +146,25 @@ public class ReaderSchemaOrchestrator implements VectorSource {
       doSmoothedProjection(readerSchema);
     } else {
       switch(scanOrchestrator.scanProj.projectionType()) {
-      case EMPTY:
-      case EXPLICIT:
-        doExplicitProjection(readerSchema);
-        break;
-      case SCHEMA_WILDCARD:
-      case STRICT_SCHEMA_WILDCARD:
-        doStrictWildcardProjection(readerSchema);
-        break;
-      case WILDCARD:
-        doWildcardProjection(readerSchema);
-        break;
-      default:
-        throw new IllegalStateException(scanOrchestrator.scanProj.projectionType().toString());
+        case EMPTY:
+        case EXPLICIT:
+          doExplicitProjection(readerSchema);
+          break;
+        case SCHEMA_WILDCARD:
+        case STRICT_SCHEMA_WILDCARD:
+          doStrictWildcardProjection(readerSchema);
+          break;
+        case WILDCARD:
+          doWildcardProjection(readerSchema);
+          break;
+        default:
+          throw new IllegalStateException(scanOrchestrator.scanProj.projectionType().toString());
       }
     }
 
     // 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();
@@ -186,7 +181,6 @@ public class ReaderSchemaOrchestrator implements VectorSource {
    * Query contains a wildcard. The schema-level projection includes
    * all columns provided by the reader.
    */
-
   private void doWildcardProjection(TupleMetadata tableSchema) {
     rootTuple = newRootTuple();
     new WildcardProjection(scanOrchestrator.scanProj,
@@ -203,7 +197,7 @@ public class ReaderSchemaOrchestrator implements VectorSource {
     return new ResolvedRow(new NullBuilderBuilder()
         .setNullType(scanOrchestrator.options.nullType)
         .allowRequiredNullColumns(scanOrchestrator.options.allowRequiredNullColumns)
-        .setOutputSchema(scanOrchestrator.options.outputSchema())
+        .setOutputSchema(scanOrchestrator.options.providedSchema())
         .build());
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ResolvedColumn.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ResolvedColumn.java
index be0cb30..1969315 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ResolvedColumn.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ResolvedColumn.java
@@ -34,7 +34,6 @@ import org.apache.drill.exec.record.metadata.ColumnMetadata;
  * along with a serialized record batch; each operator must rediscover
  * it after deserialization.
  */
-
 public abstract class ResolvedColumn implements ColumnProjection {
 
   private final VectorSource source;
@@ -65,7 +64,6 @@ public abstract class ResolvedColumn implements ColumnProjection {
    *
    * @return the MaterializedField representation of this column
    */
-
   public abstract MaterializedField schema();
 
   public void project(ResolvedTuple dest) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ResolvedMapColumn.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ResolvedMapColumn.java
index 1ba2d77..6d5614e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ResolvedMapColumn.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ResolvedMapColumn.java
@@ -30,7 +30,6 @@ import org.apache.drill.exec.record.MaterializedField;
  * in the project list), but which does not match any column in the table.
  * This kind of column gives rise to a map of null columns in the output.
  */
-
 public class ResolvedMapColumn extends ResolvedColumn {
 
   private final MaterializedField schema;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ResolvedNullColumn.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ResolvedNullColumn.java
index 4e4c026..47c5f8c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ResolvedNullColumn.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ResolvedNullColumn.java
@@ -27,7 +27,6 @@ import org.apache.drill.exec.record.metadata.ColumnMetadata;
  * mapping) and a null column spec (provides the information needed to create
  * the required null vectors.)
  */
-
 public class ResolvedNullColumn extends ResolvedColumn implements NullColumnSpec {
 
   private final String name;
@@ -60,14 +59,12 @@ public class ResolvedNullColumn extends ResolvedColumn implements NullColumnSpec
   public void setType(MajorType type) {
 
     // Adjust the default value if needed.
-
     if (this.type != null && type.getMinorType() != this.type.getMinorType()) {
       defaultValue = null;
     }
 
     // Update the actual type based on what the null-column
     // mechanism chose for this column.
-
     this.type = type;
   }
 
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 a117e7a..e6e3dfb 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
@@ -24,7 +24,7 @@ import org.apache.drill.common.exceptions.CustomErrorContext;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.physical.impl.scan.project.AbstractUnresolvedColumn.UnresolvedColumn;
 import org.apache.drill.exec.physical.impl.scan.project.AbstractUnresolvedColumn.UnresolvedWildcardColumn;
-import org.apache.drill.exec.physical.impl.scan.project.projSet.ProjectionSetBuilder;
+import org.apache.drill.exec.physical.resultSet.impl.ProjectionFilter;
 import org.apache.drill.exec.physical.resultSet.project.ImpliedTupleRequest;
 import org.apache.drill.exec.physical.resultSet.project.Projections;
 import org.apache.drill.exec.physical.resultSet.project.RequestedTuple;
@@ -135,7 +135,6 @@ import org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTes
  * @see {@link ImplicitColumnExplorer}, the class from which this class
  * evolved
  */
-
 public class ScanLevelProjection {
 
   /**
@@ -222,7 +221,7 @@ public class ScanLevelProjection {
   public static class Builder {
     private List<SchemaPath> projectionList;
     private final List<ScanProjectionParser> parsers = new ArrayList<>();
-    private TupleMetadata outputSchema;
+    private TupleMetadata providedSchema;
 
     /**
      * Context used with error messages.
@@ -247,12 +246,12 @@ public class ScanLevelProjection {
       return this;
     }
 
-    public Builder outputSchema(TupleMetadata outputSchema) {
-      this.outputSchema = outputSchema;
+    public Builder providedSchema(TupleMetadata providedSchema) {
+      this.providedSchema = providedSchema;
       return this;
     }
 
-    public Builder context(CustomErrorContext context) {
+    public Builder errorContext(CustomErrorContext context) {
       this.errorContext = context;
       return this;
     }
@@ -261,9 +260,9 @@ public class ScanLevelProjection {
       return new ScanLevelProjection(this);
     }
 
-    public TupleMetadata outputSchema( ) {
-      return outputSchema == null || outputSchema.size() == 0
-          ? null : outputSchema;
+    public TupleMetadata providedSchema( ) {
+      return providedSchema == null || providedSchema.size() == 0
+          ? null : providedSchema;
     }
 
     public List<SchemaPath> projectionList() {
@@ -282,7 +281,7 @@ public class ScanLevelProjection {
    */
   protected final CustomErrorContext errorContext;
   protected final List<SchemaPath> projectionList;
-  protected final TupleMetadata outputSchema;
+  protected final TupleMetadata readerSchema;
 
   // Configuration
 
@@ -307,13 +306,13 @@ public class ScanLevelProjection {
    * Projection definition passed to each reader. This is the set of
    * columns that the reader is asked to provide.
    */
-  protected RequestedTuple readerProjection;
+  protected ProjectionFilter readerProjection;
   protected ScanProjectionType projectionType;
 
   private ScanLevelProjection(Builder builder) {
     this.projectionList = builder.projectionList();
     this.parsers = builder.parsers;
-    this.outputSchema = builder.outputSchema();
+    this.readerSchema = builder.providedSchema();
     this.errorContext = builder.errorContext;
     doParse();
   }
@@ -344,7 +343,7 @@ public class ScanLevelProjection {
     return new Builder()
         .projection(projectionList)
         .parsers(parsers)
-        .outputSchema(outputSchema)
+        .providedSchema(outputSchema)
         .build();
   }
 
@@ -393,10 +392,11 @@ public class ScanLevelProjection {
     // projection. With a schema, we want the schema columns (which may
     // or may not correspond to reader columns.)
 
+    RequestedTuple rootProjection;
     if (projectionType == ScanProjectionType.EMPTY) {
-      readerProjection = ImpliedTupleRequest.NO_MEMBERS;
+      rootProjection = ImpliedTupleRequest.NO_MEMBERS;
     } else if (projectionType != ScanProjectionType.EXPLICIT) {
-      readerProjection = ImpliedTupleRequest.ALL_MEMBERS;
+      rootProjection = ImpliedTupleRequest.ALL_MEMBERS;
     } else {
       List<RequestedColumn> outputProj = new ArrayList<>();
       for (ColumnProjection col : outputCols) {
@@ -404,8 +404,10 @@ public class ScanLevelProjection {
           outputProj.add(((AbstractUnresolvedColumn) col).element());
         }
       }
-      readerProjection = Projections.build(outputProj);
+      rootProjection = Projections.build(outputProj);
     }
+    readerProjection = ProjectionFilter.filterFor(
+        rootProjection, readerSchema, errorContext);
   }
 
   /**
@@ -445,7 +447,7 @@ public class ScanLevelProjection {
     // placeholder to be filled in later with actual table columns.
     if (expanded) {
       projectionType =
-          outputSchema.booleanProperty(TupleMetadata.IS_STRICT_SCHEMA_PROP)
+          readerSchema.booleanProperty(TupleMetadata.IS_STRICT_SCHEMA_PROP)
           ? ScanProjectionType.STRICT_SCHEMA_WILDCARD
           : ScanProjectionType.SCHEMA_WILDCARD;
     } else if (wildcardPosn != -1) {
@@ -454,7 +456,7 @@ public class ScanLevelProjection {
   }
 
   private boolean expandOutputSchema() {
-    if (outputSchema == null) {
+    if (readerSchema == null) {
       return false;
     }
 
@@ -463,8 +465,8 @@ public class ScanLevelProjection {
     // Take the projection type from the output column's data type. That is,
     // INT[] is projected as ARRAY, etc.
 
-    for (int i = 0; i < outputSchema.size(); i++) {
-      ColumnMetadata col = outputSchema.metadata(i);
+    for (int i = 0; i < readerSchema.size(); i++) {
+      ColumnMetadata col = readerSchema.metadata(i);
 
       // Skip columns tagged as "special"; those that should not expand
       // automatically.
@@ -520,8 +522,8 @@ public class ScanLevelProjection {
 
   private void addTableColumn(RequestedColumn inCol) {
     ColumnMetadata outputCol = null;
-    if (outputSchema != null) {
-      outputCol = outputSchema.metadata(inCol.name());
+    if (readerSchema != null) {
+      outputCol = readerSchema.metadata(inCol.name());
     }
     addTableColumn(new UnresolvedColumn(inCol, outputCol));
   }
@@ -593,16 +595,11 @@ public class ScanLevelProjection {
 
   public RequestedTuple rootProjection() { return outputProjection; }
 
-  public ProjectionSetBuilder projectionSet() {
-    return new ProjectionSetBuilder()
-      .outputSchema(outputSchema)
-      .parsedProjection(readerProjection)
-      .errorContext(errorContext);
-  }
+  public ProjectionFilter readerProjection() { return readerProjection; }
 
-  public boolean hasOutputSchema() { return outputSchema != null; }
+  public boolean hasReaderSchema() { return readerSchema != null; }
 
-  public TupleMetadata outputSchema() { return outputSchema; }
+  public TupleMetadata readerSchema() { return readerSchema; }
 
   @Override
   public String toString() {
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 5626d1f..b1b3398 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
@@ -32,7 +32,6 @@ import org.apache.drill.exec.physical.impl.scan.ScanOperatorEvents;
 import org.apache.drill.exec.physical.impl.scan.ScanOperatorExec;
 import org.apache.drill.exec.physical.impl.scan.project.ReaderLevelProjection.ReaderProjectionResolver;
 import org.apache.drill.exec.physical.impl.scan.project.ScanLevelProjection.ScanProjectionParser;
-import org.apache.drill.exec.physical.impl.scan.project.projSet.TypeConverter;
 import org.apache.drill.exec.physical.resultSet.impl.ResultVectorCacheImpl;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
@@ -42,7 +41,7 @@ import org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTes
 /**
  * Performs projection of a record reader, along with a set of static
  * columns, to produce the final "public" result set (record batch)
- * for the scan operator. Primarily solve the "vector permanence"
+ * for the scan operator. Primarily solves the "vector permanence"
  * problem: that the scan operator must present the same set of vectors
  * to downstream operators despite the fact that the scan operator hosts
  * a series of readers, each of which builds its own result set.
@@ -150,7 +149,6 @@ import org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTes
  * <li>For implicit and null columns, the output vector is identical
  * to the input vector.</li>
  */
-
 public class ScanSchemaOrchestrator {
 
   public static final int MIN_BATCH_BYTE_SIZE = 256 * 1024;
@@ -170,7 +168,7 @@ public class ScanSchemaOrchestrator {
     private boolean useSchemaSmoothing;
     private boolean allowRequiredNullColumns;
     private List<SchemaPath> projection;
-    private final TypeConverter.Builder typeConverterBuilder = TypeConverter.builder();
+    private TupleMetadata providedSchema;
 
     /**
      * Option that enables whether the scan operator starts with an empty
@@ -185,7 +183,6 @@ public class ScanSchemaOrchestrator {
      * those issues are fixed. Of course, do enable the feature if you want
      * to track down the DRILL-7305 bugs.
      */
-
     private boolean enableSchemaBatch;
 
     /**
@@ -203,13 +200,11 @@ public class ScanSchemaOrchestrator {
      * disable this option only if we cannot find or fix empty-batch
      * bugs.
      */
-
     public boolean disableEmptyResults;
 
     /**
      * Context for error messages.
      */
-
     private CustomErrorContext errorContext;
 
     /**
@@ -221,8 +216,7 @@ public class ScanSchemaOrchestrator {
      * @param metadataMgr the application-specific metadata manager to use
      * for this scan
      */
-
-    public void withMetadata(MetadataManager metadataMgr) {
+    public void withImplicitColumns(MetadataManager metadataMgr) {
       metadataManager = metadataMgr;
       schemaResolvers.add(metadataManager.resolver());
     }
@@ -234,13 +228,12 @@ public class ScanSchemaOrchestrator {
      *
      * @param batchRecordLimit maximum records per batch
      */
-
-    public void setBatchRecordLimit(int batchRecordLimit) {
+    public void batchRecordLimit(int batchRecordLimit) {
       scanBatchRecordLimit = Math.max(1,
           Math.min(batchRecordLimit, ValueVector.MAX_ROW_COUNT));
     }
 
-    public void setBatchByteLimit(int byteLimit) {
+    public void batchByteLimit(int byteLimit) {
       scanBatchByteLimit = Math.max(MIN_BATCH_BYTE_SIZE,
           Math.min(byteLimit, MAX_BATCH_BYTE_SIZE));
     }
@@ -252,8 +245,7 @@ public class ScanSchemaOrchestrator {
      *
      * @param nullType the type to use for null columns
      */
-
-    public void setNullType(MajorType nullType) {
+    public void nullType(MajorType nullType) {
       this.nullType = nullType;
     }
 
@@ -263,7 +255,6 @@ public class ScanSchemaOrchestrator {
      *
      * @param flag true to enable schema smoothing, false to disable
      */
-
     public void enableSchemaSmoothing(boolean flag) {
       useSchemaSmoothing = flag;
    }
@@ -280,7 +271,7 @@ public class ScanSchemaOrchestrator {
       schemaResolvers.add(resolver);
     }
 
-    public void setProjection(List<SchemaPath> projection) {
+    public void projection(List<SchemaPath> projection) {
       this.projection = projection;
     }
 
@@ -292,11 +283,15 @@ public class ScanSchemaOrchestrator {
       disableEmptyResults = option;
     }
 
-    public TypeConverter.Builder typeConverterBuilder() {
-      return typeConverterBuilder;
+    public void providedSchema(TupleMetadata providedSchema) {
+      this.providedSchema = providedSchema;
+    }
+
+    public TupleMetadata providedSchema() {
+      return providedSchema;
     }
 
-    public void setContext(CustomErrorContext context) {
+    public void errorContext(CustomErrorContext context) {
       this.errorContext = context;
     }
 
@@ -323,7 +318,6 @@ public class ScanSchemaOrchestrator {
      * Custom null type, if provided by the operator. If
      * not set, the null type is the Drill default.
      */
-
     public final MajorType nullType;
     public final int scanBatchRecordLimit;
     public final int scanBatchByteLimit;
@@ -335,13 +329,12 @@ public class ScanSchemaOrchestrator {
      * as a plug-in rather than by copying code or subclassing this
      * mechanism.
      */
-
     public final List<ReaderProjectionResolver> schemaResolvers;
 
     public final List<SchemaPath> projection;
     public final boolean useSchemaSmoothing;
     public final boolean allowRequiredNullColumns;
-    public final TypeConverter typeConverter;
+    public final TupleMetadata providedSchema;
 
     /**
      * Context for error messages.
@@ -357,14 +350,12 @@ public class ScanSchemaOrchestrator {
       projection = builder.projection;
       useSchemaSmoothing = builder.useSchemaSmoothing;
       context = builder.errorContext;
-      typeConverter = builder.typeConverterBuilder
-        .errorContext(builder.errorContext)
-        .build();
+      providedSchema = builder.providedSchema;
       allowRequiredNullColumns = builder.allowRequiredNullColumns;
     }
 
-    protected TupleMetadata outputSchema() {
-      return typeConverter == null ? null : typeConverter.providedSchema();
+    protected TupleMetadata providedSchema() {
+      return providedSchema;
     }
   }
 
@@ -376,7 +367,6 @@ public class ScanSchemaOrchestrator {
   /**
    * Creates the metadata (file and directory) columns, if needed.
    */
-
   public final MetadataManager metadataManager;
 
   // Internal state
@@ -389,7 +379,6 @@ public class ScanSchemaOrchestrator {
    * If the Project operator ever changes so that it depends on looking up
    * vectors rather than vector instances, this cache can be deprecated.
    */
-
   protected final ResultVectorCacheImpl vectorCache;
   protected final ScanLevelProjection scanProj;
   private ReaderSchemaOrchestrator currentReader;
@@ -407,7 +396,6 @@ public class ScanSchemaOrchestrator {
 
     // If no metadata manager was provided, create a mock
     // version just to keep code simple.
-
     if (builder.metadataManager == null) {
       metadataManager = new NoOpMetadataManager();
     } else {
@@ -422,18 +410,18 @@ public class ScanSchemaOrchestrator {
 
     ScanProjectionParser parser = metadataManager.projectionParser();
     if (parser != null) {
+
       // Insert in last position to expand wildcards at
       // the end of the tuple.
       options.parsers.add(parser);
     }
 
     // Parse the projection list.
-
     scanProj = ScanLevelProjection.builder()
         .projection(options.projection)
         .parsers(options.parsers)
-        .outputSchema(options.outputSchema())
-        .context(builder.errorContext())
+        .providedSchema(options.providedSchema())
+        .errorContext(builder.errorContext())
         .build();
     if (scanProj.projectAll() && options.useSchemaSmoothing) {
       schemaSmoother = new SchemaSmoother(scanProj, options.schemaResolvers);
@@ -442,7 +430,6 @@ public class ScanSchemaOrchestrator {
     }
 
     // Build the output container.
-
     outputContainer = new VectorContainer(allocator);
   }
 
@@ -460,6 +447,13 @@ public class ScanSchemaOrchestrator {
     return currentReader != null && currentReader.hasSchema();
   }
 
+  /**
+   * Returns the provided reader schema.
+   */
+  public TupleMetadata providedSchema() {
+    return options.providedSchema();
+  }
+
   public VectorContainer output() {
     return outputContainer;
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/StaticColumnLoader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/StaticColumnLoader.java
index 8499b68..68d274b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/StaticColumnLoader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/StaticColumnLoader.java
@@ -19,7 +19,7 @@ package org.apache.drill.exec.physical.impl.scan.project;
 
 import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
 import org.apache.drill.exec.physical.resultSet.ResultVectorCache;
-import org.apache.drill.exec.physical.resultSet.impl.OptionBuilder;
+import org.apache.drill.exec.physical.resultSet.impl.ResultSetOptionBuilder;
 import org.apache.drill.exec.physical.resultSet.impl.ResultSetLoaderImpl;
 import org.apache.drill.exec.record.VectorContainer;
 
@@ -36,8 +36,8 @@ public abstract class StaticColumnLoader {
 
   public StaticColumnLoader(ResultVectorCache vectorCache) {
 
-    ResultSetLoaderImpl.ResultSetOptions options = new OptionBuilder()
-          .setVectorCache(vectorCache)
+    ResultSetLoaderImpl.ResultSetOptions options = new ResultSetOptionBuilder()
+          .vectorCache(vectorCache)
           .build();
     loader = new ResultSetLoaderImpl(vectorCache.allocator(), options);
     this.vectorCache = vectorCache;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/AbstractProjectionSet.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/AbstractProjectionSet.java
deleted file mode 100644
index ec12e0b..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/AbstractProjectionSet.java
+++ /dev/null
@@ -1,80 +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.project.projSet;
-
-import org.apache.drill.common.exceptions.CustomErrorContext;
-import org.apache.drill.exec.physical.resultSet.ProjectionSet;
-import org.apache.drill.exec.record.metadata.ColumnMetadata;
-import org.apache.drill.exec.record.metadata.TupleMetadata;
-import org.apache.drill.exec.vector.accessor.convert.ColumnConversionFactory;
-
-/**
- * Base class for projection set implementations. Handles an optional
- * type conversion based on a provided schema, custom conversion, or both.
- */
-
-public abstract class AbstractProjectionSet implements ProjectionSet {
-  protected final TypeConverter typeConverter;
-  protected final TupleMetadata providedSchema;
-  protected final boolean isStrict;
-  protected CustomErrorContext errorContext;
-
-  public AbstractProjectionSet(TypeConverter typeConverter) {
-    this.typeConverter = typeConverter;
-    providedSchema = typeConverter == null ? null :
-        typeConverter.providedSchema();
-    isStrict = providedSchema != null &&
-        typeConverter.providedSchema().booleanProperty(TupleMetadata.IS_STRICT_SCHEMA_PROP);
-  }
-
-  public AbstractProjectionSet(TypeConverter typeConverter, boolean isStrict) {
-    this.typeConverter = typeConverter;
-    providedSchema = typeConverter == null ? null :
-        typeConverter.providedSchema();
-    this.isStrict = isStrict;
-  }
-
-  public AbstractProjectionSet() {
-    this(null);
-  }
-
-  @Override
-  public void setErrorContext(CustomErrorContext errorContext) {
-    this.errorContext = errorContext;
-  }
-
-  protected static boolean isSpecial(ColumnMetadata col) {
-    return col.booleanProperty(ColumnMetadata.EXCLUDE_FROM_WILDCARD);
-  }
-
-  protected ColumnMetadata outputSchema(ColumnMetadata col) {
-    return providedSchema == null ? null :
-      providedSchema.metadata(col.name());
-  }
-
-  protected ColumnConversionFactory conversion(ColumnMetadata inputSchema, ColumnMetadata outputCol) {
-    return typeConverter == null ? null :
-      typeConverter.conversionFactory(inputSchema, outputCol);
-  }
-
-  protected TypeConverter childConverter(ColumnMetadata outputSchema) {
-    TupleMetadata childSchema = outputSchema == null ? null : outputSchema.tupleSchema();
-    return typeConverter == null ? null :
-      typeConverter.childConverter(childSchema);
-  }
-}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/AbstractReadColProj.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/AbstractReadColProj.java
deleted file mode 100644
index d3fc2ee..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/AbstractReadColProj.java
+++ /dev/null
@@ -1,46 +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.project.projSet;
-
-import org.apache.drill.exec.physical.resultSet.ProjectionSet;
-import org.apache.drill.exec.physical.resultSet.ProjectionSet.ColumnReadProjection;
-import org.apache.drill.exec.record.metadata.ColumnMetadata;
-import org.apache.drill.exec.vector.accessor.convert.ColumnConversionFactory;
-
-public abstract class AbstractReadColProj implements ColumnReadProjection {
-  protected final ColumnMetadata readSchema;
-
-  public AbstractReadColProj(ColumnMetadata readSchema) {
-    this.readSchema = readSchema;
-  }
-
-  @Override
-  public ColumnMetadata readSchema() { return readSchema; }
-
-  @Override
-  public boolean isProjected() { return true; }
-
-  @Override
-  public ColumnConversionFactory conversionFactory() { return null; }
-
-  @Override
-  public ColumnMetadata providedSchema() { return readSchema; }
-
-  @Override
-  public ProjectionSet mapProjection() { return ProjectionSetFactory.projectAll(); }
-}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/EmptyProjectionSet.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/EmptyProjectionSet.java
deleted file mode 100644
index d02a04c..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/EmptyProjectionSet.java
+++ /dev/null
@@ -1,51 +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.project.projSet;
-
-import org.apache.drill.common.exceptions.CustomErrorContext;
-import org.apache.drill.exec.physical.resultSet.ProjectionSet;
-import org.apache.drill.exec.record.metadata.ColumnMetadata;
-
-/**
- * Handles simple cases in which either all columns are projected
- * or no columns are projected.
- */
-
-public class EmptyProjectionSet implements ProjectionSet {
-
-  public static final ProjectionSet PROJECT_NONE = new EmptyProjectionSet();
-
-  @Override
-  public ColumnReadProjection readProjection(ColumnMetadata col) {
-    return new UnprojectedReadColumn(col);
-  }
-
-  @Override
-  public ColumnReadProjection readDictProjection(ColumnMetadata col) {
-    return readProjection(col);
-  }
-
-  @Override
-  public void setErrorContext(CustomErrorContext errorContext) { }
-
-  @Override
-  public boolean isEmpty() { return true; }
-
-  @Override
-  public boolean isProjected(String colName) { return false; }
-}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ExplicitProjectionSet.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ExplicitProjectionSet.java
deleted file mode 100644
index aace7e3..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ExplicitProjectionSet.java
+++ /dev/null
@@ -1,118 +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.project.projSet;
-
-import org.apache.drill.exec.physical.resultSet.ProjectionSet;
-import org.apache.drill.exec.physical.resultSet.project.RequestedColumn;
-import org.apache.drill.exec.physical.resultSet.project.RequestedColumnImpl;
-import org.apache.drill.exec.physical.resultSet.project.RequestedTuple;
-import org.apache.drill.exec.physical.resultSet.project.RequestedTuple.TupleProjectionType;
-import org.apache.drill.exec.record.metadata.ColumnMetadata;
-import org.apache.drill.exec.vector.accessor.convert.ColumnConversionFactory;
-import org.apache.drill.exec.vector.complex.DictVector;
-
-/**
- * Projection set based on an explicit set of columns provided
- * in the physical plan. Columns in the list are projected, others
- * are not.
- */
-
-public class ExplicitProjectionSet extends AbstractProjectionSet {
-
-  private final RequestedTuple requestedProj;
-
-  public ExplicitProjectionSet(RequestedTuple requestedProj, TypeConverter typeConverter) {
-    super(typeConverter);
-    this.requestedProj = requestedProj;
-  }
-
-  @Override
-  public boolean isProjected(String colName) {
-    return requestedProj.get(colName) != null;
-  }
-
-  @Override
-  public ColumnReadProjection readProjection(ColumnMetadata col) {
-    RequestedColumn reqCol = requestedProj.get(col.name());
-    if (reqCol == null) {
-      return new UnprojectedReadColumn(col);
-    }
-
-    return getReadProjection(col, reqCol);
-  }
-
-  private ColumnReadProjection getReadProjection(ColumnMetadata col, RequestedColumn reqCol) {
-    ColumnMetadata outputSchema = outputSchema(col);
-    ProjectionChecker.validateProjection(reqCol, outputSchema == null ? col : outputSchema, errorContext);
-    if (!col.isMap() && !col.isDict()) {
-
-      // Non-map column.
-
-      ColumnConversionFactory conv = conversion(col, outputSchema);
-      return new ProjectedReadColumn(col, reqCol, outputSchema, conv);
-    } else {
-
-      // Maps are tuples. Create a tuple projection and wrap it in
-      // a column projection.
-
-      TypeConverter childConverter = childConverter(outputSchema);
-      ProjectionSet mapProjection;
-      if (! reqCol.isTuple() || reqCol.tuple().type() == TupleProjectionType.ALL) {
-
-        // Projection is simple: "m". This is equivalent to
-        // (non-SQL) m.*
-        // This may also be a projection of the form m.a, m. The
-        // general projection takes precedence.
-
-        mapProjection =  new WildcardProjectionSet(childConverter, isStrict);
-      } else {
-
-        // Else, selected map items are projected, say m.a, m.c.
-        // (Here, we'll never hit the case where none of the map is
-        // projected; that case, while allowed in the RequestedTuple
-        // implementation, can never occur in a SELECT list.)
-
-        mapProjection = new ExplicitProjectionSet(reqCol.tuple(), childConverter);
-      }
-      if (col.isMap()) {
-        return new ProjectedMapColumn(col, reqCol, outputSchema, mapProjection);
-      } else {
-        return new ProjectedDictColumn(col, reqCol, outputSchema, mapProjection);
-      }
-    }
-  }
-
-  @Override
-  public ColumnReadProjection readDictProjection(ColumnMetadata col) {
-    // Unlike for a MAP, requestedProj contains a key value, rather than nested field's name:
-    // create DICT's members somewhat artificially
-
-    assert DictVector.fieldNames.contains(col.name());
-    if (col.name().equals(DictVector.FIELD_KEY_NAME)) {
-      // This field is considered not projected but its
-      // vector and writer will be instantiated later.
-      return new UnprojectedReadColumn(col);
-    }
-
-    RequestedColumn reqCol = new RequestedColumnImpl(requestedProj, col.name()); // this is the 'value' column
-    return getReadProjection(col, reqCol);
-  }
-
-  @Override
-  public boolean isEmpty() { return requestedProj.projections().isEmpty(); }
-}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectedDictColumn.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectedDictColumn.java
deleted file mode 100644
index 6805a0e..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectedDictColumn.java
+++ /dev/null
@@ -1,39 +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.project.projSet;
-
-import org.apache.drill.exec.physical.resultSet.ProjectionSet;
-import org.apache.drill.exec.physical.resultSet.project.RequestedColumn;
-import org.apache.drill.exec.record.metadata.ColumnMetadata;
-
-public class ProjectedDictColumn extends ProjectedReadColumn {
-
-  private final ProjectionSet tupleProjection;
-
-  public ProjectedDictColumn(ColumnMetadata readSchema,
-                            RequestedColumn requestedCol, ColumnMetadata outputSchema,
-                            ProjectionSet tupleProjection) {
-    super(readSchema, requestedCol, outputSchema, null);
-    this.tupleProjection = tupleProjection;
-  }
-
-  @Override
-  public ProjectionSet mapProjection() {
-    return tupleProjection;
-  }
-}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectedMapColumn.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectedMapColumn.java
deleted file mode 100644
index f29ca31..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectedMapColumn.java
+++ /dev/null
@@ -1,39 +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.project.projSet;
-
-import org.apache.drill.exec.physical.resultSet.ProjectionSet;
-import org.apache.drill.exec.physical.resultSet.project.RequestedColumn;
-import org.apache.drill.exec.record.metadata.ColumnMetadata;
-
-public class ProjectedMapColumn extends ProjectedReadColumn {
-
-  private final ProjectionSet mapProjection;
-
-  public ProjectedMapColumn(ColumnMetadata readSchema,
-      RequestedColumn requestedCol, ColumnMetadata outputSchema,
-      ProjectionSet mapProjection) {
-    super(readSchema, requestedCol, outputSchema, null);
-    this.mapProjection = mapProjection;
-  }
-
-  @Override
-  public ProjectionSet mapProjection() {
-    return mapProjection;
-  }
-}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectedReadColumn.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectedReadColumn.java
deleted file mode 100644
index 834b657..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectedReadColumn.java
+++ /dev/null
@@ -1,73 +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.project.projSet;
-
-import org.apache.drill.exec.physical.resultSet.ProjectionSet;
-import org.apache.drill.exec.physical.resultSet.project.RequestedColumn;
-import org.apache.drill.exec.record.metadata.ColumnMetadata;
-import org.apache.drill.exec.vector.accessor.convert.ColumnConversionFactory;
-
-/**
- * Projected column. Includes at least the reader schema. May also
- * include projection specification, and output schema and a type
- * conversion.
- */
-
-public class ProjectedReadColumn extends AbstractReadColProj {
-  @SuppressWarnings("unused")
-  private final RequestedColumn requestedCol;
-  private final ColumnMetadata outputSchema;
-  private final ColumnConversionFactory conversionFactory;
-
-  public ProjectedReadColumn(ColumnMetadata readSchema) {
-    this(readSchema, null, null, null);
-  }
-
-  public ProjectedReadColumn(ColumnMetadata readSchema,
-      RequestedColumn requestedCol) {
-    this(readSchema, requestedCol, null, null);
-  }
-
-  public ProjectedReadColumn(ColumnMetadata readSchema,
-      ColumnMetadata outputSchema, ColumnConversionFactory conversionFactory) {
-    this(readSchema, null, outputSchema, null);
-  }
-
-  public ProjectedReadColumn(ColumnMetadata readSchema,
-      RequestedColumn requestedCol, ColumnMetadata outputSchema,
-      ColumnConversionFactory conversionFactory) {
-    super(readSchema);
-    this.requestedCol = requestedCol;
-    this.outputSchema = outputSchema;
-    this.conversionFactory = conversionFactory;
-  }
-
-  @Override
-  public ColumnMetadata providedSchema() {
-    return outputSchema == null ? readSchema : outputSchema;
-  }
-
-  @Override
-  public ProjectionSet mapProjection() {
-    // Should never occur: maps should use the map class.
-    return null;
-  }
-
-  @Override
-  public ColumnConversionFactory conversionFactory() { return conversionFactory; }
-}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectionSetBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectionSetBuilder.java
deleted file mode 100644
index 6f788b6..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectionSetBuilder.java
+++ /dev/null
@@ -1,101 +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.project.projSet;
-
-import java.util.Collection;
-
-import org.apache.drill.common.exceptions.CustomErrorContext;
-import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.exec.physical.resultSet.ProjectionSet;
-import org.apache.drill.exec.physical.resultSet.project.Projections;
-import org.apache.drill.exec.physical.resultSet.project.RequestedTuple;
-import org.apache.drill.exec.physical.resultSet.project.RequestedTuple.TupleProjectionType;
-import org.apache.drill.exec.record.metadata.TupleMetadata;
-
-public class ProjectionSetBuilder {
-
-  private RequestedTuple parsedProjection;
-  private TypeConverter typeConverter;
-  private CustomErrorContext errorContext;
-
-  /**
-   * Record (batch) readers often read a subset of available table columns,
-   * but want to use a writer schema that includes all columns for ease of
-   * writing. (For example, a CSV reader must read all columns, even if the user
-   * wants a subset. The unwanted columns are simply discarded.)
-   * <p>
-   * This option provides a projection list, in the form of column names, for
-   * those columns which are to be projected. Only those columns will be
-   * backed by value vectors; non-projected columns will be backed by "null"
-   * writers that discard all values.
-   *
-   * @param projection the list of projected columns
-   * @return this builder
-   */
-
-  public ProjectionSetBuilder projectionList(Collection<SchemaPath> projection) {
-    if (projection == null) {
-      parsedProjection = null;
-    } else {
-      parsedProjection = Projections.parse(projection);
-    }
-    return this;
-  }
-
-  public ProjectionSetBuilder parsedProjection(RequestedTuple projection) {
-    parsedProjection = projection;
-    return this;
-  }
-
-  public ProjectionSetBuilder outputSchema(TupleMetadata schema) {
-    typeConverter = TypeConverter.builder().providedSchema(schema).build();
-    return this;
-  }
-
-  public ProjectionSetBuilder typeConverter(TypeConverter converter) {
-    this.typeConverter = converter;
-    return this;
-  }
-
-  public ProjectionSetBuilder errorContext(CustomErrorContext errorContext) {
-    this.errorContext = errorContext;
-    return this;
-  }
-
-  public ProjectionSet build() {
-    TupleProjectionType projType = parsedProjection == null ?
-        TupleProjectionType.ALL : parsedProjection.type();
-
-    ProjectionSet projSet;
-    switch (projType) {
-      case ALL:
-        projSet = new WildcardProjectionSet(typeConverter);
-        break;
-      case NONE:
-        projSet = ProjectionSetFactory.projectNone();
-        break;
-      case SOME:
-        projSet = new ExplicitProjectionSet(parsedProjection, typeConverter);
-        break;
-      default:
-        throw new IllegalStateException("Unexpected projection type: " + projType.toString());
-    }
-    projSet.setErrorContext(errorContext);
-    return projSet;
-  }
-}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectionSetFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectionSetFactory.java
deleted file mode 100644
index 46e1b94..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectionSetFactory.java
+++ /dev/null
@@ -1,79 +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.project.projSet;
-
-import java.util.List;
-import java.util.Map;
-
-import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.exec.physical.impl.scan.project.projSet.TypeConverter.CustomTypeTransform;
-import org.apache.drill.exec.physical.resultSet.ProjectionSet;
-import org.apache.drill.exec.physical.resultSet.project.Projections;
-import org.apache.drill.exec.physical.resultSet.project.RequestedTuple;
-import org.apache.drill.exec.record.metadata.ColumnMetadata;
-import org.apache.drill.exec.vector.accessor.convert.ColumnConversionFactory;
-import org.apache.drill.exec.vector.accessor.convert.StandardConversions.ConversionDefn;
-
-public class ProjectionSetFactory {
-
-  private static class SimpleTransform implements CustomTypeTransform {
-
-    private final ColumnConversionFactory colFactory;
-
-    public SimpleTransform(ColumnConversionFactory colFactory) {
-      this.colFactory = colFactory;
-    }
-
-    @Override
-    public ColumnConversionFactory transform(ColumnMetadata inputDefn,
-        Map<String, String> properties,
-        ColumnMetadata outputDefn, ConversionDefn defn) {
-      return colFactory;
-    }
-  }
-
-  public static ProjectionSet projectAll() { return new WildcardProjectionSet(null); }
-
-  public static ProjectionSet projectNone() { return EmptyProjectionSet.PROJECT_NONE; }
-
-
-  public static ProjectionSet wrap(RequestedTuple mapProjection) {
-    switch (mapProjection.type()) {
-    case ALL:
-      return projectAll();
-    case NONE:
-      return projectNone();
-    case SOME:
-      return new ExplicitProjectionSet(mapProjection, null);
-    default:
-      throw new IllegalStateException("Unexpected projection type: " +
-            mapProjection.type().toString());
-    }
-  }
-
-  public static ProjectionSet build(List<SchemaPath> selection) {
-    if (selection == null) {
-      return projectAll();
-    }
-    return wrap(Projections.parse(selection));
-  }
-
-  public static CustomTypeTransform simpleTransform(ColumnConversionFactory colFactory) {
-    return new SimpleTransform(colFactory);
-  }
-}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/TypeConverter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/TypeConverter.java
deleted file mode 100644
index 8269116..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/TypeConverter.java
+++ /dev/null
@@ -1,175 +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.project.projSet;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.drill.common.exceptions.CustomErrorContext;
-import org.apache.drill.common.exceptions.UserException;
-import org.apache.drill.exec.record.metadata.ColumnMetadata;
-import org.apache.drill.exec.record.metadata.TupleMetadata;
-import org.apache.drill.exec.vector.accessor.convert.ColumnConversionFactory;
-import org.apache.drill.exec.vector.accessor.convert.StandardConversions;
-import org.apache.drill.exec.vector.accessor.convert.StandardConversions.ConversionDefn;
-import org.apache.drill.exec.vector.accessor.convert.StandardConversions.ConversionType;
-
-public class TypeConverter {
-  private static final org.slf4j.Logger logger =
-      org.slf4j.LoggerFactory.getLogger(TypeConverter.class);
-
-  public static interface CustomTypeTransform {
-    ColumnConversionFactory transform(ColumnMetadata inputDefn,
-        Map<String, String> properties,
-        ColumnMetadata outputDefn, ConversionDefn defn);
-  }
-
-  private static class NullTypeTransform implements CustomTypeTransform {
-    @Override
-    public ColumnConversionFactory transform(ColumnMetadata inputDefn,
-        Map<String, String> properties,
-        ColumnMetadata outputDefn, ConversionDefn defn) {
-      return null;
-    }
-  }
-
-  public static class Builder {
-    private TupleMetadata providedSchema;
-    private CustomTypeTransform transform;
-    private Map<String, String> properties;
-    private CustomErrorContext errorContext;
-
-    public Builder providedSchema(TupleMetadata schema) {
-      providedSchema = schema;
-      return this;
-    }
-
-    public TupleMetadata providedSchema() { return providedSchema; }
-
-    public Builder transform(TypeConverter.CustomTypeTransform transform) {
-      this.transform = transform;
-      return this;
-    }
-
-    public Builder properties(Map<String, String> properties) {
-      this.properties = properties;
-      return this;
-    }
-
-    public Builder setConversionProperty(String key, String value) {
-      if (key == null || value == null) {
-        return this;
-      }
-      if (properties == null) {
-        properties = new HashMap<>();
-      }
-      properties.put(key, value);
-      return this;
-    }
-
-    public Builder errorContext(CustomErrorContext errorContext) {
-      this.errorContext = errorContext;
-      return this;
-    }
-
-    public TypeConverter build() {
-      return new TypeConverter(this);
-    }
-  }
-
-  private final TupleMetadata providedSchema;
-  private final CustomTypeTransform customTransform;
-  private final Map<String, String> properties;
-  private final CustomErrorContext errorContext;
-
-  public static Builder builder() { return new Builder(); }
-
-  public TypeConverter(Builder builder) {
-    this.providedSchema = builder.providedSchema;
-    this.customTransform = builder.transform == null ?
-        new NullTypeTransform() : builder.transform;
-    this.properties = builder.properties;
-    this.errorContext = builder.errorContext;
-  }
-
-  public TypeConverter(TypeConverter parent,
-      TupleMetadata childSchema) {
-    this.providedSchema = childSchema;
-    this.customTransform = parent.customTransform;
-    this.properties = parent.properties;
-    this.errorContext = parent.errorContext;
-  }
-
-  public TupleMetadata providedSchema() { return providedSchema; }
-
-  public ColumnConversionFactory conversionFactory(ColumnMetadata inputSchema,
-      ColumnMetadata outputCol) {
-    if (outputCol == null) {
-      return customConversion(inputSchema);
-    } else {
-      return schemaBasedConversion(inputSchema, outputCol);
-    }
-  }
-
-  private ColumnConversionFactory customConversion(ColumnMetadata inputSchema) {
-    return customTransform.transform(inputSchema, properties, null, null);
-  }
-
-  public ColumnConversionFactory schemaBasedConversion(ColumnMetadata inputSchema,
-      ColumnMetadata outputCol) {
-
-    // Custom transforms take priority. Allows replacing the standard
-    // conversions. Also allows conversions between the same type, such
-    // as rescaling units.
-
-    ConversionDefn defn = StandardConversions.analyze(inputSchema, outputCol);
-    ColumnConversionFactory factory = customTransform.transform(inputSchema, properties, outputCol, defn);
-    if (factory != null) {
-      return factory;
-    }
-
-    // Some conversions are automatic.
-
-    if (defn.type != ConversionType.EXPLICIT) {
-      return null;
-    }
-
-    // If an explicit conversion is needed, but no standard conversion
-    // is available, we have no way to do the conversion.
-
-    if (defn.conversionClass == null) {
-      throw UserException.validationError()
-        .message("Runtime type conversion not available")
-        .addContext("Input type", inputSchema.typeString())
-        .addContext("Output type", outputCol.typeString())
-        .addContext(errorContext)
-        .build(logger);
-    }
-
-    // Return a factory for the conversion.
-
-    return StandardConversions.factory(defn.conversionClass, properties);
-  }
-
-  public TypeConverter childConverter(TupleMetadata childSchema) {
-    if (childSchema == null && providedSchema == null) {
-      return this;
-    }
-    return new TypeConverter(this, childSchema);
-  }
-}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/UnprojectedReadColumn.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/UnprojectedReadColumn.java
deleted file mode 100644
index b34ecb6..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/UnprojectedReadColumn.java
+++ /dev/null
@@ -1,41 +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.project.projSet;
-
-import org.apache.drill.exec.physical.resultSet.ProjectionSet;
-import org.apache.drill.exec.record.metadata.ColumnMetadata;
-
-/**
- * Unprojected column. No validation needed. No type conversion.
- * Reader column just "free wheels", without a materialized vector,
- * accepting any data the reader cares to throw at it, then simply
- * discarding that data.
- */
-
-public class UnprojectedReadColumn extends AbstractReadColProj {
-
-  public UnprojectedReadColumn(ColumnMetadata readSchema) {
-    super(readSchema);
-  }
-
-  @Override
-  public boolean isProjected() { return false; }
-
-  @Override
-  public ProjectionSet mapProjection() { return ProjectionSetFactory.projectNone(); }
-}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/WildcardProjectionSet.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/WildcardProjectionSet.java
deleted file mode 100644
index f7d216f..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/WildcardProjectionSet.java
+++ /dev/null
@@ -1,70 +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.project.projSet;
-
-import org.apache.drill.exec.record.metadata.ColumnMetadata;
-import org.apache.drill.exec.vector.accessor.convert.ColumnConversionFactory;
-
-public class WildcardProjectionSet extends AbstractProjectionSet {
-
-  public WildcardProjectionSet(TypeConverter typeConverter) {
-    super(typeConverter);
-  }
-
-  public WildcardProjectionSet(TypeConverter typeConverter, boolean isStrict) {
-    super(typeConverter, isStrict);
-  }
-
-  @Override
-  public boolean isProjected(String colName) { return true; }
-
-  @Override
-  public ColumnReadProjection readProjection(ColumnMetadata col) {
-    if (isSpecial(col)) {
-      return new UnprojectedReadColumn(col);
-    }
-    ColumnMetadata outputSchema = outputSchema(col);
-    if (outputSchema == null) {
-      if (isStrict) {
-        return new UnprojectedReadColumn(col);
-      }
-    } else if (isSpecial(outputSchema)) {
-      return new UnprojectedReadColumn(col);
-    }
-    if (col.isMap() || col.isDict()) {
-      return new ProjectedMapColumn(col, null, outputSchema,
-          new WildcardProjectionSet(childConverter(outputSchema), isStrict));
-
-    } else {
-      ColumnConversionFactory conv = conversion(col, outputSchema);
-      return new ProjectedReadColumn(col, null, outputSchema, conv);
-    }
-  }
-
-  @Override
-  public ColumnReadProjection readDictProjection(ColumnMetadata col) {
-    return readProjection(col);
-  }
-
-  // Wildcard means use whatever schema is provided by the reader,
-  // so the projection itself is non-empty even if the reader has no
-  // columns.
-
-  @Override
-  public boolean isEmpty() { return false; }
-}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/package-info.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/package-info.java
deleted file mode 100644
index 55af6c1..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/package-info.java
+++ /dev/null
@@ -1,99 +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.
- */
-
-/**
- * The dynamic projection in Drill is complex. With the advent of
- * provided schema, we now have many ways to manage projection. The
- * classes here implement these many policies. They are implemented
- * as distinct classes (rather than chains of if-statements) to
- * make the classes easier to test and reason about.
- * <p>
- * Projection is a combination of three distinct policies:
- * <ul>
- * <li>Projection policy (all, none, explicit, etc.)</li>
- * <li>Column policy (unprojected, explicit projection,
- * projection with schema, etc.)</li>
- * <li>Type conversion: none, based on a provided schema,
- * custom.</li>
- * </ul>
- * Experience has shown that these must be separated: each is designed
- * and tested separately to keep the problem tractable.
- *
- * <h4>Projection Set Cases</h4>
- *
- * The project cases and their classes:
- * <p>
- * <dl>
- * <dt>{@link EmptyProjectionSet}</dt>
- * <dd><tt>SELECT COUNT(*)</tt>: Project nothing. Only count records.</dd>
- * <dl>
- * <dt>{@link WildcardProjectionSet}</dt>
- * <dd><tt>SELECT *</tt>: Project everything, with an optional provided
- * schema. If a schema is provided, and is strict, then project only
- * reader columns that appear in the provided schema.
- * However, don't project columns which have been marked as
- * special: {@link ColumnMetadata#EXCLUDE_FROM_WILDCARD}, whether marked
- * in the reader or provided schemas.</dd>
- * <dt>{@link ExplicitProjectionSet}</dt>
- * <dd><tt>SELECT a, b[10], c.d</tt>: Explicit projection with or without
- * a schema. Project only the selected columns. Verify that the reader
- * provides column types/modes consistent with the implied form in the
- * projection list. That is, in this example, `b` must be an array.</dd>
- * </dl>
- *
- * <h4>Column Projection Cases</h4>
- *
- * Each projection set answers a query: "the reader wants to add such-and-so
- * column: what should I do?" Since the reader is free to add any column,
- * we don't cache the list of columns as is done with the parsed project
- * list, or the output schema. Instead, we handle each column on a
- * case-by-case basis; we create a {@link ColumnReadProjection} instance
- * to answer the query. Instances of this class are meant to be transient:
- * use them and discard them. We answer the query differently depending on
- * many factors, including:
- * <p>
- * <dl>
- * <dt>{@link UnprojectedReadColumn}</dt>
- * <dd>Column is not projected. Nothing to convert, no type checks
- * needed. The result set loader should create a dummy writer for this
- * case.</dd>
- * <dt>{@link ProjectedReadColumn}</dt>
- * <dd>Column is projected. It may have an associated projection list
- * item, an output schema, or a type conversion. All these variations
- * should be transparent to the consumer.</dd>
- * </dl>
- *
- * <h4>Type Conversion</h4>
- *
- * The {@link TypeConverter} class handles a provided schema, custom type
- * conversion, and custom properties passed to the conversion shims. A null
- * type converter passed to a projection set means no conversion is done.
- * (The mechanism creates a dummy projection in this case.)
- *
- * <h4>Construction</h4>
- *
- * Two classes build the above complex cases:
- * <p>
- * <dl>
- * <dt>{@link ProjectionSetFactory}<dt>
- * <dd>Builds simple projection sets that take few parameters.</dd>
- * <dt>{@link ProjectionSetBuilder}</dt>
- * <dd>Handles the complex cases.</dd>
- */
-
-package org.apache.drill.exec.physical.impl.scan.project.projSet;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/ProjectionSet.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/ProjectionSet.java
deleted file mode 100644
index 81b6a24..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/ProjectionSet.java
+++ /dev/null
@@ -1,96 +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.resultSet;
-
-import org.apache.drill.common.exceptions.CustomErrorContext;
-import org.apache.drill.exec.record.metadata.ColumnMetadata;
-import org.apache.drill.exec.vector.accessor.convert.ColumnConversionFactory;
-
-/**
- * Provides a dynamic, run-time view of a projection set. Used by
- * the result set loader to:
- * <ul>
- * <li>Determine if a column is projected according to some
- * defined projection schema (see implementation for details.)</li>
- * <li>Provide type conversions, either using built-in implicit
- * conversions, or a custom conversion. Type conversions require
- * the reader column and a "provided" column that gives the "to"
- * type for the conversion. Without the "to" column, the reader
- * column type is used as-is.</li>
- * <li>Verify that the (possibly converted) type and mode are
- * compatible with an explicit projection item. For example, if
- * the query has `a.b`, but `a` is scalar, then there is an
- * inconsistency.</li>
- * </ul>
- * <p>
- * This interface filters columns added dynamically
- * at scan time. The reader may offer a column (as to add a column
- * writer for the column.) The projection mechanism says whether to
- * materialize the column, or whether to ignore the column and
- * return a dummy column writer.
- * <p>
- * The Project All must handle several additional nuances:
- * <ul>
- * <li>External schema: If an external schema is provided, then that
- * schema may be "strict" which causes the wildcard to expand to the
- * set of columns defined within the schema. When used with columns
- * added dynamically, a column may be excluded from the projection
- * set if it is not part of the defined external schema.</ul>
- * <li>Metadata filtering: A reader may offer a special column which
- * is available only in explicit projection, and behaves like Drill's
- * implicit file columns. Such columns are not included in a "project
- * all" projection.</li>
- * <p>
- * At present, only the top-level row supports these additional filtering
- * options; they are not supported on maps (though could be with additional
- * effort.)
- * <p>
- * Special columns are generic and thus handled here. External schema
- * is handled in a subclass in the scan projection framework.
- * <p>
- */
-public interface ProjectionSet {
-
-  /**
-   * Response to a query against a reader projection to indicate projection
-   * status of a reader-provided column. This is a transient object which
-   * indicates whether a reader column is projected, and if so, the attributes
-   * of that projection.
-   */
-  public interface ColumnReadProjection {
-
-    /**
-     * Determine if the given column is to be projected. Used when
-     * adding columns to the result set loader. Skips columns omitted
-     * from an explicit projection, or columns within a wildcard projection
-     * where the column is "special" and is not expanded in the wildcard.
-     */
-    boolean isProjected();
-
-    ColumnMetadata readSchema();
-    ColumnMetadata providedSchema();
-    ColumnConversionFactory conversionFactory();
-    ProjectionSet mapProjection();
-  }
-
-  boolean isProjected(String colName);
-  void setErrorContext(CustomErrorContext errorContext);
-  ColumnReadProjection readProjection(ColumnMetadata col);
-  ColumnReadProjection readDictProjection(ColumnMetadata col);
-  boolean isEmpty();
-}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/ResultSetLoader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/ResultSetLoader.java
index 5fd3164..e75f4cc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/ResultSetLoader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/ResultSetLoader.java
@@ -38,7 +38,6 @@ import org.apache.drill.exec.vector.complex.impl.VectorContainerWriter;
  * @see {@link VectorContainerWriter}, the class which this class
  * replaces
  */
-
 public interface ResultSetLoader {
 
   public static final int DEFAULT_ROW_COUNT = BaseValueVector.INITIAL_VALUE_ALLOCATION;
@@ -46,15 +45,13 @@ public interface ResultSetLoader {
   /**
    * Context for error messages.
    */
-
-  CustomErrorContext context();
+  CustomErrorContext errorContext();
 
   /**
    * Current schema version. The version increments by one each time
    * a column is added.
    * @return the current schema version
    */
-
   int schemaVersion();
 
   /**
@@ -63,7 +60,6 @@ public interface ResultSetLoader {
    *
    * @param count target batch row count
    */
-
   void setTargetRowCount(int count);
 
   /**
@@ -72,7 +68,6 @@ public interface ResultSetLoader {
    *
    * @return the target row count for batches that this loader produces
    */
-
   int targetRowCount();
 
   /**
@@ -84,7 +79,6 @@ public interface ResultSetLoader {
    * in-flight batch, even if the batch has not yet reached the target
    * row count
    */
-
   int targetVectorSize();
 
   /**
@@ -93,7 +87,6 @@ public interface ResultSetLoader {
    * @return the number of batches produced including the current
    * one
    */
-
   int batchCount();
 
   /**
@@ -101,7 +94,6 @@ public interface ResultSetLoader {
    * current batch.
    * @return total row count
    */
-
   int totalRowCount();
 
   /**
@@ -112,14 +104,12 @@ public interface ResultSetLoader {
    * @return true if at least one row is available to harvest, false
    * otherwise
    */
-
   boolean hasRows();
 
   /**
    * Start a new row batch. Valid only when first started, or after the
    * previous batch has been harvested.
    */
-
   void startBatch();
 
   /**
@@ -129,7 +119,6 @@ public interface ResultSetLoader {
    *
    * @return writer for the top-level columns
    */
-
   RowSetLoader writer();
 
   /**
@@ -140,7 +129,6 @@ public interface ResultSetLoader {
    * @return true if the client can add a row to the loader, false if
    * not
    */
-
   boolean writeable();
 
   /**
@@ -160,7 +148,6 @@ public interface ResultSetLoader {
    * @param values column values in column index order
    * @return this loader
    */
-
   ResultSetLoader setRow(Object...values);
 
   /**
@@ -179,7 +166,6 @@ public interface ResultSetLoader {
    *         requested amount. If less, the client should call this method for
    *         multiple batches until the requested count is reached
    */
-
   int skipRows(int requestedCount);
 
   /**
@@ -199,7 +185,6 @@ public interface ResultSetLoader {
    * @return true if no columns are actually projected, false if at
    * least one column is projected
    */
-
   boolean isProjectionEmpty();
 
   /**
@@ -211,7 +196,6 @@ public interface ResultSetLoader {
    *
    * @return container used to publish results from this loader
    */
-
   VectorContainer outputContainer();
 
   /**
@@ -233,7 +217,6 @@ public interface ResultSetLoader {
    *
    * @return the row batch to send downstream
    */
-
   VectorContainer harvest();
 
   /**
@@ -243,7 +226,6 @@ public interface ResultSetLoader {
    * @return the extended schema of the harvested batch which includes
    * any allocation hints used when creating the batch
    */
-
   TupleMetadata harvestSchema();
 
   /**
@@ -252,7 +234,6 @@ public interface ResultSetLoader {
    *
    * @return real or dummy vector cache
    */
-
   ResultVectorCache vectorCache();
 
   /**
@@ -260,6 +241,5 @@ public interface ResultSetLoader {
    * available, or the caller wishes to cancel the current row batch
    * and complete.
    */
-
   void close();
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/ResultVectorCache.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/ResultVectorCache.java
index c723812..1dee608 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/ResultVectorCache.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/ResultVectorCache.java
@@ -30,7 +30,7 @@ import org.apache.drill.exec.vector.ValueVector;
 
 public interface ResultVectorCache {
   BufferAllocator allocator();
-  ValueVector addOrGet(MaterializedField colSchema);
+  ValueVector vectorFor(MaterializedField colSchema);
   MajorType getType(String name);
   boolean isPermissive();
   ResultVectorCache childCache(String colName);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ColumnBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ColumnBuilder.java
index 3f8abce..3c5df73 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ColumnBuilder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ColumnBuilder.java
@@ -22,7 +22,6 @@ import java.util.ArrayList;
 import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.common.types.Types;
-import org.apache.drill.exec.physical.resultSet.ProjectionSet.ColumnReadProjection;
 import org.apache.drill.exec.physical.resultSet.impl.ColumnState.PrimitiveColumnState;
 import org.apache.drill.exec.physical.resultSet.impl.ListState.ListVectorState;
 import org.apache.drill.exec.physical.resultSet.impl.RepeatedListState.RepeatedListColumnState;
@@ -32,6 +31,7 @@ import org.apache.drill.exec.physical.resultSet.impl.SingleVectorState.SimpleVec
 import org.apache.drill.exec.physical.resultSet.impl.TupleState.MapArrayState;
 import org.apache.drill.exec.physical.resultSet.impl.TupleState.MapColumnState;
 import org.apache.drill.exec.physical.resultSet.impl.TupleState.MapVectorState;
+import org.apache.drill.exec.physical.resultSet.impl.TupleState.SingleDictState;
 import org.apache.drill.exec.physical.resultSet.impl.TupleState.SingleMapState;
 import org.apache.drill.exec.physical.resultSet.impl.UnionState.UnionColumnState;
 import org.apache.drill.exec.physical.resultSet.impl.UnionState.UnionVectorState;
@@ -87,7 +87,6 @@ import org.apache.drill.exec.vector.complex.UnionVector;
  * to be added empty, then the members to be added one by one. See
  * {@link BuildFromSchema} for the class that builds up a compound structure.
  */
-
 public class ColumnBuilder {
 
   /**
@@ -102,30 +101,17 @@ public class ColumnBuilder {
    */
   public ColumnState buildColumn(ContainerState parent, ColumnMetadata columnSchema) {
 
-    ColumnReadProjection colProj;
-    if (parent instanceof TupleState.DictState) {
-      colProj = parent.projectionSet().readDictProjection(columnSchema);
-    } else {
-      colProj = parent.projectionSet().readProjection(columnSchema);
-    }
-    switch (colProj.providedSchema().structureType()) {
-    case DICT:
-      return buildDict(parent, colProj);
-    case TUPLE:
-      return buildMap(parent, colProj);
-    case VARIANT:
-      // Variant: UNION or (non-repeated) LIST
-      if (columnSchema.isArray()) {
-        // (non-repeated) LIST (somewhat like a repeated UNION)
-        return buildList(parent, colProj);
-      } else {
-        // (Non-repeated) UNION
-        return buildUnion(parent, colProj);
-      }
-    case MULTI_ARRAY:
-      return buildRepeatedList(parent, colProj);
-    default:
-      return buildPrimitive(parent, colProj);
+    switch (columnSchema.structureType()) {
+      case DICT:
+        return buildDict(parent, columnSchema);
+      case TUPLE:
+        return buildMap(parent, columnSchema);
+      case VARIANT:
+        return buildVariant(parent, columnSchema);
+      case MULTI_ARRAY:
+        return buildRepeatedList(parent, columnSchema);
+      default:
+        return buildPrimitive(parent, columnSchema);
     }
   }
 
@@ -139,37 +125,30 @@ public class ColumnBuilder {
    * @param colProj implied projection type for the column
    * @return column state for the new column
    */
+  private ColumnState buildPrimitive(ContainerState parent, ColumnMetadata columnSchema) {
 
-  private ColumnState buildPrimitive(ContainerState parent, ColumnReadProjection colProj) {
-    ColumnMetadata columnSchema = colProj.providedSchema();
-
-    ValueVector vector;
-    if (!colProj.isProjected() && !allowCreation(parent)) {
-
-      // Column is not projected. No materialized backing for the column.
-
-      vector = null;
-    } else {
+    final ValueVector vector;
+    if (parent.projection().isProjected(columnSchema) || allowCreation(parent)) {
 
       // Create the vector for the column.
-
-      vector = parent.vectorCache().addOrGet(columnSchema.schema());
+      vector = parent.vectorCache().vectorFor(columnSchema.schema());
 
       // In permissive mode, the mode or precision of the vector may differ
       // from that requested. Update the schema to match.
-
-      if (parent.vectorCache().isPermissive() && ! vector.getField().isEquivalent(columnSchema.schema())) {
+      if (parent.vectorCache().isPermissive() && !vector.getField().isEquivalent(columnSchema.schema())) {
         columnSchema = ((PrimitiveColumnMetadata) columnSchema).mergeWith(vector.getField());
       }
+    } else {
+
+      // Column is not projected. No materialized backing for the column.
+      vector = null;
     }
 
     // Create the writer.
-
-    final AbstractObjectWriter colWriter = ColumnWriterFactory.buildColumnWriter(
-        columnSchema, colProj.conversionFactory(), vector);
+    final AbstractObjectWriter colWriter =
+        ColumnWriterFactory.buildColumnWriter(columnSchema, vector);
 
     // Build the vector state which manages the vector.
-
     VectorState vectorState;
     if (vector == null) {
       vectorState = new NullVectorState();
@@ -184,9 +163,7 @@ public class ColumnBuilder {
     }
 
     // Create the column state which binds the vector and writer together.
-
-    return new PrimitiveColumnState(parent.loader(), colWriter,
-        vectorState);
+    return new PrimitiveColumnState(parent.loader(), colWriter, vectorState);
   }
 
   /**
@@ -201,7 +178,7 @@ public class ColumnBuilder {
    * @return {@code true} if the parent is {@code DICT} and its {@code value} is accessed by key
    */
   private boolean allowCreation(ContainerState parent) {
-    return parent instanceof TupleState.DictState && !parent.projectionSet().isEmpty();
+    return parent instanceof TupleState.DictState && !parent.projection().isEmpty();
   }
 
   /**
@@ -214,88 +191,82 @@ public class ColumnBuilder {
    * @param colProj implied projection type for the column
    * @return column state for the map column
    */
-
-  private ColumnState buildMap(ContainerState parent, ColumnReadProjection colProj) {
-    ColumnMetadata columnSchema = colProj.providedSchema();
+  private ColumnState buildMap(ContainerState parent, ColumnMetadata columnSchema) {
 
     // When dynamically adding columns, must add the (empty)
     // map by itself, then add columns to the map via separate
     // calls.
-
     assert columnSchema.isMap();
     assert columnSchema.tupleSchema().isEmpty();
 
     // Create the vector, vector state and writer.
-
     if (columnSchema.isArray()) {
-      return buildMapArray(parent, colProj);
+      return buildMapArray(parent, columnSchema);
     } else {
-      return buildSingleMap(parent, colProj);
+      return buildSingleMap(parent, columnSchema);
     }
   }
 
-  private ColumnState buildSingleMap(ContainerState parent, ColumnReadProjection colProj) {
-    ColumnMetadata columnSchema = colProj.providedSchema();
+  private ColumnState buildSingleMap(ContainerState parent, ColumnMetadata columnSchema) {
 
-    MapVector vector;
-    VectorState vectorState;
-    if (!colProj.isProjected()) {
-      vector = null;
-      vectorState = new NullVectorState();
-    } else {
+    final ProjectionFilter projFilter = parent.projection();
+    final boolean isProjected = projFilter.isProjected(columnSchema);
+
+    final MapVector vector;
+    final VectorState vectorState;
+    if (isProjected) {
 
       // Don't get the map vector from the vector cache. Map vectors may
       // have content that varies from batch to batch. Only the leaf
       // vectors can be cached.
-
       assert columnSchema.tupleSchema().isEmpty();
       vector = new MapVector(columnSchema.schema(), parent.loader().allocator(), null);
       vectorState = new MapVectorState(vector, new NullVectorState());
+    } else {
+      vector = null;
+      vectorState = new NullVectorState();
     }
     final TupleObjectWriter mapWriter = MapWriter.buildMap(columnSchema, vector, new ArrayList<>());
     final SingleMapState mapState = new SingleMapState(parent.loader(),
         parent.vectorCache().childCache(columnSchema.name()),
-        colProj.mapProjection());
+        projFilter.mapProjection(isProjected, columnSchema.name()));
     return new MapColumnState(mapState, mapWriter, vectorState, parent.isVersioned());
   }
 
-  private ColumnState buildMapArray(ContainerState parent, ColumnReadProjection colProj) {
-    ColumnMetadata columnSchema = colProj.providedSchema();
+  private ColumnState buildMapArray(ContainerState parent, ColumnMetadata columnSchema) {
 
-    // Create the map's offset vector.
+    final ProjectionFilter projFilter = parent.projection();
+    final boolean isProjected = projFilter.isProjected(columnSchema);
 
-    RepeatedMapVector mapVector;
-    UInt4Vector offsetVector;
-    if (!colProj.isProjected()) {
-      mapVector = null;
-      offsetVector = null;
-    } else {
+    // Create the map's offset vector.
+    final RepeatedMapVector mapVector;
+    final UInt4Vector offsetVector;
+    if (isProjected) {
 
       // Creating the map vector will create its contained vectors if we
       // give it a materialized field with children. So, instead pass a clone
       // without children so we can add them.
-
       final ColumnMetadata mapColSchema = columnSchema.cloneEmpty();
 
       // Don't get the map vector from the vector cache. Map vectors may
       // have content that varies from batch to batch. Only the leaf
       // vectors can be cached.
-
       assert columnSchema.tupleSchema().isEmpty();
       mapVector = new RepeatedMapVector(mapColSchema.schema(),
           parent.loader().allocator(), null);
       offsetVector = mapVector.getOffsetVector();
+    } else {
+      mapVector = null;
+      offsetVector = null;
     }
 
     // Create the writer using the offset vector
-
     final AbstractObjectWriter writer = MapWriter.buildMapArray(
         columnSchema, mapVector, new ArrayList<>());
 
     // Wrap the offset vector in a vector state
-
     VectorState offsetVectorState;
-    if (!colProj.isProjected()) {
+    if (!projFilter.isProjected(columnSchema)) {
       offsetVectorState = new NullVectorState();
     } else {
       offsetVectorState = new OffsetVectorState(
@@ -306,13 +277,24 @@ public class ColumnBuilder {
     final VectorState mapVectorState = new MapVectorState(mapVector, offsetVectorState);
 
     // Assemble it all into the column state.
-
     final MapArrayState mapState = new MapArrayState(parent.loader(),
         parent.vectorCache().childCache(columnSchema.name()),
-        colProj.mapProjection());
+        projFilter.mapProjection(isProjected, columnSchema.name()));
     return new MapColumnState(mapState, writer, mapVectorState, parent.isVersioned());
   }
 
+  private ColumnState buildVariant(ContainerState parent,
+      ColumnMetadata columnSchema) {
+    // Variant: UNION or (non-repeated) LIST
+    if (columnSchema.isArray()) {
+      // (non-repeated) LIST (somewhat like a repeated UNION)
+      return buildList(parent, columnSchema);
+    } else {
+      // (Non-repeated) UNION
+      return buildUnion(parent, columnSchema);
+    }
+  }
+
   /**
    * Builds a union column.
    * <p>
@@ -330,58 +312,49 @@ public class ColumnBuilder {
    * @param colProj column schema
    * @return column
    */
-  private ColumnState buildUnion(ContainerState parent, ColumnReadProjection colProj) {
-    ColumnMetadata columnSchema = colProj.providedSchema();
-    assert columnSchema.isVariant() && ! columnSchema.isArray();
+  private ColumnState buildUnion(ContainerState parent, ColumnMetadata columnSchema) {
+     assert columnSchema.isVariant() && ! columnSchema.isArray();
 
     // Create the union vector.
     // Don't get the union vector from the vector cache. Union vectors may
     // have content that varies from batch to batch. Only the leaf
     // vectors can be cached.
-
     assert columnSchema.variantSchema().size() == 0;
     final UnionVector vector = new UnionVector(columnSchema.schema(), parent.loader().allocator(), null);
 
     // Then the union writer.
-
     final UnionWriterImpl unionWriter = new UnionWriterImpl(columnSchema, vector, null);
     final VariantObjectWriter writer = new VariantObjectWriter(unionWriter);
 
     // The union vector state which manages the types vector.
-
     final UnionVectorState vectorState = new UnionVectorState(vector, unionWriter);
 
     // Create the manager for the columns within the union.
-
     final UnionState unionState = new UnionState(parent.loader(),
         parent.vectorCache().childCache(columnSchema.name()));
 
     // Bind the union state to the union writer to handle column additions.
-
     unionWriter.bindListener(unionState);
 
     // Assemble it all into a union column state.
-
     return new UnionColumnState(parent.loader(), writer, vectorState, unionState);
   }
 
-  private ColumnState buildList(ContainerState parent, ColumnReadProjection colProj) {
-    ColumnMetadata columnSchema = colProj.providedSchema();
+  private ColumnState buildList(ContainerState parent, ColumnMetadata columnSchema) {
 
     // If the list has declared a single type, and has indicated that this
     // is the only type expected, then build the list as a nullable array
     // of that type. Else, build the list as array of (a possibly empty)
     // union.
-
     final VariantMetadata variant = columnSchema.variantSchema();
     if (variant.isSimple()) {
       if (variant.size() == 1) {
-        return buildSimpleList(parent, colProj);
+        return buildSimpleList(parent, columnSchema);
       } else if (variant.size() == 0) {
         throw new IllegalArgumentException("Size of a non-expandable list can't be zero");
       }
     }
-    return buildUnionList(parent, colProj);
+    return buildUnionList(parent, columnSchema);
   }
 
   /**
@@ -391,52 +364,43 @@ public class ColumnBuilder {
    * <p>
    * List vectors (lists of optional values) are not supported in
    * Drill. The code here works up through the scan operator. But, other operators do
-   * not support the <tt>ListVector</tt> type.
+   * not support the {@code ListVector</tt> type.
    *
    * @param parent the parent (tuple, union or list) that holds this list
    * @param colProj metadata description of the list which must contain
    * exactly one subtype
    * @return the column state for the list
    */
-
-  private ColumnState buildSimpleList(ContainerState parent, ColumnReadProjection colProj) {
-    ColumnMetadata columnSchema = colProj.providedSchema();
+  private ColumnState buildSimpleList(ContainerState parent, ColumnMetadata columnSchema) {
 
     // The variant must have the one and only type.
-
     assert columnSchema.variantSchema().size() == 1;
     assert columnSchema.variantSchema().isSimple();
 
     // Create the manager for the one and only column within the list.
-
     final ListState listState = new ListState(parent.loader(),
         parent.vectorCache().childCache(columnSchema.name()));
 
     // Create the child vector, writer and state.
-
     final ColumnMetadata memberSchema = columnSchema.variantSchema().listSubtype();
     final ColumnState memberState = buildColumn(listState, memberSchema);
     listState.setSubColumn(memberState);
 
     // Create the list vector. Contains a single type.
-
     final ListVector listVector = new ListVector(columnSchema.schema().cloneEmpty(),
         parent.loader().allocator(), null);
     listVector.setChildVector(memberState.vector());
 
     // Create the list writer: an array of the one type.
-
     final ListWriterImpl listWriter = new ListWriterImpl(columnSchema,
         listVector, memberState.writer());
     final AbstractObjectWriter listObjWriter = new ArrayObjectWriter(listWriter);
 
     // Create the list vector state that tracks the list vector lifecycle.
-
     final ListVectorState vectorState = new ListVectorState(listWriter,
         memberState.writer().events(), listVector);
 
     // Assemble it all into a union column state.
-
     return new UnionColumnState(parent.loader(),
         listObjWriter, vectorState, listState);
   }
@@ -450,23 +414,19 @@ public class ColumnBuilder {
    * <p>
    * List vectors (lists of unions) are not supported in
    * Drill. The code here works up through the scan operator. But, other operators do
-   * not support the <tt>ListVector</tt> type.
+   * not support the {@code ListVector} type.
    *
    * @param parent the parent (tuple, union or list) that holds this list
    * @param colProj metadata description of the list (must be empty of
    * subtypes)
    * @return the column state for the list
    */
-
-  private ColumnState buildUnionList(ContainerState parent, ColumnReadProjection colProj) {
-    ColumnMetadata columnSchema = colProj.providedSchema();
+  private ColumnState buildUnionList(ContainerState parent, ColumnMetadata columnSchema) {
 
     // The variant must start out empty.
-
     assert columnSchema.variantSchema().size() == 0;
 
     // Create the union writer, bound to an empty list shim.
-
     final UnionWriterImpl unionWriter = new UnionWriterImpl(columnSchema);
     unionWriter.bindShim(new EmptyListShim());
     final VariantObjectWriter unionObjWriter = new VariantObjectWriter(unionWriter);
@@ -476,49 +436,40 @@ public class ColumnBuilder {
     // Don't get the list vector from the vector cache. List vectors may
     // have content that varies from batch to batch. Only the leaf
     // vectors can be cached.
-
     final ListVector listVector = new ListVector(columnSchema.schema(),
         parent.loader().allocator(), null);
 
     // Create the list vector state that tracks the list vector lifecycle.
-
     final ListVectorState vectorState = new ListVectorState(unionWriter, listVector);
 
     // Create the list writer: an array of unions.
-
     final AbstractObjectWriter listWriter = new ArrayObjectWriter(
         new ListWriterImpl(columnSchema, listVector, unionObjWriter));
 
     // Create the manager for the columns within the list (which may or
     // may not be grouped into a union.)
-
     final ListState listState = new ListState(parent.loader(),
         parent.vectorCache().childCache(columnSchema.name()));
 
     // Bind the union state to the union writer to handle column additions.
-
     unionWriter.bindListener(listState);
 
     // Assemble it all into a union column state.
-
     return new UnionColumnState(parent.loader(),
         listWriter, vectorState, listState);
   }
 
   private ColumnState buildRepeatedList(ContainerState parent,
-      ColumnReadProjection colProj) {
-    ColumnMetadata columnSchema = colProj.providedSchema();
+      ColumnMetadata columnSchema) {
 
     assert columnSchema.type() == MinorType.LIST;
     assert columnSchema.mode() == DataMode.REPEATED;
 
     // The schema provided must be empty. The caller must add
     // the element type after creating the repeated writer itself.
-
     assert columnSchema.childSchema() == null;
 
     // Build the repeated vector.
-
     final RepeatedListVector vector = new RepeatedListVector(
         columnSchema.emptySchema(), parent.loader().allocator(), null);
 
@@ -526,70 +477,59 @@ public class ColumnBuilder {
     // incrementally because it might be complex (a map or another
     // repeated list.) To start, use a dummy to avoid need for if-statements
     // everywhere.
-
     final ColumnMetadata dummyElementSchema = new PrimitiveColumnMetadata(
         MaterializedField.create(columnSchema.name(),
             Types.repeated(MinorType.NULL)));
     final AbstractObjectWriter dummyElement = ColumnWriterFactory.buildDummyColumnWriter(dummyElementSchema);
 
     // Create the list writer: an array of arrays.
-
     final AbstractObjectWriter arrayWriter = RepeatedListWriter.buildRepeatedList(
         columnSchema, vector, dummyElement);
 
     // Create the list vector state that tracks the list vector lifecycle.
     // For a repeated list, we only care about
-
     final RepeatedListVectorState vectorState = new RepeatedListVectorState(
         arrayWriter, vector);
 
     // Build the container that tracks the array contents
-
     final RepeatedListState listState = new RepeatedListState(
         parent.loader(),
         parent.vectorCache().childCache(columnSchema.name()));
 
     // Bind the list state as the list event listener.
-
     ((RepeatedListWriter) arrayWriter.array()).bindListener(listState);
 
     // Assemble it all into a column state. This state will
     // propagate events down to the (one and only) child state.
-
     return new RepeatedListColumnState(parent.loader(),
         arrayWriter, vectorState, listState);
   }
 
-  private ColumnState buildDict(ContainerState parent, ColumnReadProjection colProj) {
-    ColumnMetadata columnSchema = colProj.providedSchema();
+  private ColumnState buildDict(ContainerState parent, ColumnMetadata columnSchema) {
 
     // When dynamically adding columns, must add the (empty)
     // dict by itself, then add columns to the dict via separate
     // calls (the same way as is done for MAP).
-
     assert columnSchema.isDict();
     assert columnSchema.tupleSchema().isEmpty();
 
     // Create the vector, vector state and writer.
-
     if (columnSchema.isArray()) {
-      return buildDictArray(parent, colProj);
+      return buildDictArray(parent, columnSchema);
     } else {
-      return buildSingleDict(parent, colProj);
+      return buildSingleDict(parent, columnSchema);
     }
   }
 
-  private ColumnState buildDictArray(ContainerState parent, ColumnReadProjection colProj) {
-    ColumnMetadata columnSchema = colProj.providedSchema();
+  private ColumnState buildDictArray(ContainerState parent, ColumnMetadata columnSchema) {
 
-    // Create the dict's offset vector.
+    final ProjectionFilter projFilter = parent.projection();
+    final boolean isProjected = projFilter.isProjected(columnSchema);
 
-    RepeatedDictVector repeatedDictVector;
-    UInt4Vector offsetVector;
-    if (!colProj.isProjected()) {
-      repeatedDictVector = null;
-      offsetVector = null;
-    } else {
+    // Create the dict's offset vector.
+    final RepeatedDictVector repeatedDictVector;
+    final UInt4Vector offsetVector;
+    if (isProjected) {
 
       // Creating the dict vector will create its contained vectors if we
       // give it a materialized field with children. So, instead pass a clone
@@ -605,6 +545,9 @@ public class ColumnBuilder {
       repeatedDictVector = new RepeatedDictVector(dictColMetadata.schema(),
           parent.loader().allocator(), null);
       offsetVector = repeatedDictVector.getOffsetVector();
+    } else {
+      repeatedDictVector = null;
+      offsetVector = null;
     }
 
     // Create the writer using the offset vector
@@ -616,7 +559,7 @@ public class ColumnBuilder {
 
     VectorState offsetVectorState;
     VectorState dictOffsetVectorState;
-    if (!colProj.isProjected()) {
+    if (!projFilter.isProjected(columnSchema)) {
       offsetVectorState = new NullVectorState();
       dictOffsetVectorState = new NullVectorState();
     } else {
@@ -634,39 +577,37 @@ public class ColumnBuilder {
         new TupleState.DictArrayVectorState(repeatedDictVector, offsetVectorState, dictOffsetVectorState);
 
     // Assemble it all into the column state.
-
     final TupleState.DictArrayState dictArrayState = new TupleState.DictArrayState(parent.loader(),
         parent.vectorCache().childCache(columnSchema.name()),
-        colProj.mapProjection());
+        projFilter.mapProjection(isProjected, columnSchema.name()));
     return new TupleState.DictColumnState(
         dictArrayState, writer, mapVectorState, parent.isVersioned());
   }
 
-  private ColumnState buildSingleDict(ContainerState parent, ColumnReadProjection colProj) {
-    ColumnMetadata columnSchema = colProj.providedSchema();
+  private ColumnState buildSingleDict(ContainerState parent, ColumnMetadata columnSchema) {
 
-    // Create the dict's offset vector.
+    final ProjectionFilter projFilter = parent.projection();
+    final boolean isProjected = projFilter.isProjected(columnSchema);
 
-    DictVector dictVector;
-    UInt4Vector offsetVector;
-    if (!colProj.isProjected()) {
-      dictVector = null;
-      offsetVector = null;
-    } else {
+    // Create the dict's offset vector.
+    final DictVector dictVector;
+    final UInt4Vector offsetVector;
+    if (isProjected) {
 
       // Creating the dict vector will create its contained vectors if we
       // give it a materialized field with children. So, instead pass a clone
       // without children so we can add them.
-
       final ColumnMetadata dictColMetadata = columnSchema.cloneEmpty();
 
       // Don't get the dict vector from the vector cache. Dict vectors may
       // have content that varies from batch to batch. Only the leaf
       // vectors can be cached.
-
       assert columnSchema.tupleSchema().isEmpty();
       dictVector = new DictVector(dictColMetadata.schema(), parent.loader().allocator(), null);
       offsetVector = dictVector.getOffsetVector();
+    } else {
+      dictVector = null;
+      offsetVector = null;
     }
 
     // Create the writer using the offset vector
@@ -675,8 +616,8 @@ public class ColumnBuilder {
 
     // Wrap the offset vector in a vector state
 
-    VectorState offsetVectorState;
-    if (!colProj.isProjected()) {
+    final VectorState offsetVectorState;
+    if (!projFilter.isProjected(columnSchema)) {
       offsetVectorState = new NullVectorState();
     } else {
       offsetVectorState = new OffsetVectorState(
@@ -687,9 +628,8 @@ public class ColumnBuilder {
     final VectorState mapVectorState = new TupleState.SingleDictVectorState(dictVector, offsetVectorState);
 
     // Assemble it all into the column state.
-
-    final TupleState.SingleDictState dictArrayState = new TupleState.SingleDictState(parent.loader(), parent.vectorCache().childCache(columnSchema.name()),
-        colProj.mapProjection());
+    final SingleDictState dictArrayState = new SingleDictState(parent.loader(), parent.vectorCache().childCache(columnSchema.name()),
+        projFilter.mapProjection(isProjected, columnSchema.name()));
     return new TupleState.DictColumnState(
         dictArrayState, writer, mapVectorState, parent.isVersioned());
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ColumnState.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ColumnState.java
index 82c37ee..3815744 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ColumnState.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ColumnState.java
@@ -26,6 +26,8 @@ import org.apache.drill.exec.vector.accessor.impl.HierarchicalFormatter;
 import org.apache.drill.exec.vector.accessor.writer.AbstractObjectWriter;
 import org.apache.drill.exec.vector.accessor.writer.WriterEvents;
 import org.apache.drill.exec.vector.accessor.writer.WriterEvents.ColumnWriterListener;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Represents the write-time state for a column including the writer and the (optional)
@@ -40,14 +42,12 @@ import org.apache.drill.exec.vector.accessor.writer.WriterEvents.ColumnWriterLis
  * these differences.
  */
 public abstract class ColumnState {
-
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ColumnState.class);
+  private static final Logger logger = LoggerFactory.getLogger(ColumnState.class);
 
   /**
    * Primitive (non-map) column state. Handles all three cardinalities.
    * Column metadata is hosted on the writer.
    */
-
   public static class PrimitiveColumnState extends ColumnState implements ColumnWriterListener {
 
     public PrimitiveColumnState(LoaderInternals loader,
@@ -77,7 +77,6 @@ public abstract class ColumnState {
      * Get the output schema. For a primitive (non-structured) column,
      * the output schema is the same as the internal schema.
      */
-
     @Override
     public ColumnMetadata outputSchema() { return schema(); }
 
@@ -133,14 +132,12 @@ public abstract class ColumnState {
    * here: the variables for a column and the point within the column
    * lifecycle.
    */
-
   protected enum State {
 
     /**
      * Column is in the normal state of writing with no overflow
      * in effect.
      */
-
     NORMAL,
 
     /**
@@ -150,14 +147,12 @@ public abstract class ColumnState {
      * columns will be in OVERFLOW state, unwritten columns in
      * NORMAL state.
      */
-
     OVERFLOW,
 
     /**
      * Indicates that the column has data saved
      * in the overflow batch.
      */
-
     LOOK_AHEAD,
 
     /**
@@ -165,7 +160,6 @@ public abstract class ColumnState {
      * was added after overflow, so there is no vector for the column
      * in the harvested batch.
      */
-
     NEW_LOOK_AHEAD
   }
 
@@ -183,7 +177,6 @@ public abstract class ColumnState {
    * cardinality is the total number of array items in the
    * vector.
    */
-
   protected int cardinality;
   protected int outputIndex = -1;
 
@@ -191,9 +184,9 @@ public abstract class ColumnState {
       AbstractObjectWriter writer, VectorState vectorState) {
     this.loader = loader;
     this.vectorState = vectorState;
-    addVersion = writer.isProjected() ?
+    this.addVersion = writer.isProjected() ?
         loader.bumpVersion() : loader.activeSchemaVersion();
-    state = loader.hasOverflow() ? State.NEW_LOOK_AHEAD : State.NORMAL;
+    this.state = loader.hasOverflow() ? State.NEW_LOOK_AHEAD : State.NORMAL;
     this.writer = writer;
   }
 
@@ -213,34 +206,30 @@ public abstract class ColumnState {
    * batch. Restore the look-ahead buffer to the
    * active vector so we start writing where we left off.
    */
-
   public void startBatch(boolean schemaOnly) {
     switch (state) {
-    case NORMAL:
-      if (! schemaOnly) {
-        allocateVectors();
-      }
-      break;
-
-    case NEW_LOOK_AHEAD:
-
-      // Column is new, was not exchanged with backup vector
+      case NORMAL:
+        if (! schemaOnly) {
+          allocateVectors();
+        }
+        break;
 
-      break;
+      case NEW_LOOK_AHEAD:
 
-    case LOOK_AHEAD:
+        // Column is new, was not exchanged with backup vector
+        break;
 
-      // Restore the look-ahead values to the main vector.
+      case LOOK_AHEAD:
 
-      vectorState.startBatchWithLookAhead();
-      break;
+        // Restore the look-ahead values to the main vector.
+        vectorState.startBatchWithLookAhead();
+        break;
 
-    default:
-      throw new IllegalStateException("Unexpected state: " + state);
+      default:
+        throw new IllegalStateException("Unexpected state: " + state);
     }
 
     // In all cases, we are back to normal writing.
-
     state = State.NORMAL;
   }
 
@@ -271,11 +260,9 @@ public abstract class ColumnState {
     }
 
     // Otherwise, do the roll-over to a look-ahead vector.
-
     vectorState.rollover(cardinality);
 
     // Remember that we did this overflow processing.
-
     state = State.OVERFLOW;
   }
 
@@ -285,31 +272,28 @@ public abstract class ColumnState {
    * look-ahead vector and put the full vector buffer back into the active
    * vector.
    */
-
   public void harvestWithLookAhead() {
     switch (state) {
-    case NEW_LOOK_AHEAD:
-
-      // If added after overflow, no data to save from the complete
-      // batch: the vector does not appear in the completed batch.
-
-      break;
+      case NEW_LOOK_AHEAD:
 
-    case OVERFLOW:
+        // If added after overflow, no data to save from the complete
+        // batch: the vector does not appear in the completed batch.
+        break;
 
-      // Otherwise, restore the original, full buffer and
-      // last write position.
+      case OVERFLOW:
 
-      vectorState.harvestWithLookAhead();
+        // Otherwise, restore the original, full buffer and
+        // last write position.
+        vectorState.harvestWithLookAhead();
 
-      // Remember that we have look-ahead values stashed away in the
-      // backup vector.
+        // Remember that we have look-ahead values stashed away in the
+        // backup vector.
 
-      state = State.LOOK_AHEAD;
-      break;
+        state= State.LOOK_AHEAD;
+        break;
 
-    default:
-      throw new IllegalStateException("Unexpected state: " + state);
+      default:
+        throw new IllegalStateException("Unexpected state: " + state);
     }
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ContainerState.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ContainerState.java
index fde2844..3e1cab3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ContainerState.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ContainerState.java
@@ -19,8 +19,6 @@ package org.apache.drill.exec.physical.resultSet.impl;
 
 import java.util.Collection;
 
-import org.apache.drill.exec.physical.impl.scan.project.projSet.ProjectionSetFactory;
-import org.apache.drill.exec.physical.resultSet.ProjectionSet;
 import org.apache.drill.exec.physical.resultSet.ResultVectorCache;
 import org.apache.drill.exec.record.metadata.ColumnMetadata;
 
@@ -43,23 +41,23 @@ import org.apache.drill.exec.record.metadata.ColumnMetadata;
 public abstract class ContainerState {
 
   protected final LoaderInternals loader;
-  protected final ProjectionSet projectionSet;
+  protected final ProjectionFilter projectionSet;
   protected ColumnState parentColumn;
 
   /**
    * Vector cache for this loader.
-   * @see {@link OptionBuilder#setVectorCache()}.
+   * {@see ResultSetOptionBuilder#setVectorCache()}.
    */
   protected final ResultVectorCache vectorCache;
 
-  public ContainerState(LoaderInternals loader, ResultVectorCache vectorCache, ProjectionSet projectionSet) {
+  public ContainerState(LoaderInternals loader, ResultVectorCache vectorCache, ProjectionFilter projectionSet) {
     this.loader = loader;
     this.vectorCache = vectorCache;
     this.projectionSet = projectionSet;
   }
 
   public ContainerState(LoaderInternals loader, ResultVectorCache vectorCache) {
-    this(loader, vectorCache, ProjectionSetFactory.projectAll());
+    this(loader, vectorCache, ProjectionFilter.PROJECT_ALL);
   }
 
   public void bindColumnState(ColumnState parentState) {
@@ -69,19 +67,19 @@ public abstract class ContainerState {
   public abstract int innerCardinality();
   protected abstract void addColumn(ColumnState colState);
   protected abstract Collection<ColumnState> columnStates();
+  protected ProjectionFilter projection() { return projectionSet; }
 
   /**
    * Reports whether this container is subject to version management. Version
    * management adds columns to the output container at harvest time based on
    * whether they should appear in the output batch.
    *
-   * @return <tt>true</tt> if versioned
+   * @return {@code true} if versioned
    */
   protected abstract boolean isVersioned();
 
   protected LoaderInternals loader() { return loader; }
   public ResultVectorCache vectorCache() { return vectorCache; }
-  public ProjectionSet projectionSet() { return projectionSet; }
 
   public ColumnState addColumn(ColumnMetadata columnSchema) {
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/LoaderInternals.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/LoaderInternals.java
index 9f65699..9ea478a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/LoaderInternals.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/LoaderInternals.java
@@ -17,20 +17,19 @@
  */
 package org.apache.drill.exec.physical.resultSet.impl;
 
+import org.apache.drill.common.exceptions.CustomErrorContext;
 import org.apache.drill.exec.memory.BufferAllocator;
 
 /**
  * The internal view of the result set loader. Provides operations to manage the batch
  * and batch schema.
  */
-
 interface LoaderInternals {
 
   /**
    * Allocator to use when allocating buffers for vectors
    * @return buffer allocator
    */
-
   BufferAllocator allocator();
 
   /**
@@ -39,7 +38,6 @@ interface LoaderInternals {
    *
    * @return the new schema version
    */
-
   int bumpVersion();
 
   /**
@@ -56,7 +54,6 @@ interface LoaderInternals {
    * @param allocationBytes number of bytes allocated to a vector
    * in the batch setup step
    */
-
   void tallyAllocations(int allocationBytes);
 
   /**
@@ -64,10 +61,9 @@ interface LoaderInternals {
    * state occurs when a vector has become full, but before the batch
    * is harvested.
    *
-   * @return <tt>true</tt> if an overflow has occurred in the present
+   * @return {@code true</tt> if an overflow has occurred in the present
    * row
    */
-
   boolean hasOverflow();
 
   /**
@@ -78,7 +74,6 @@ interface LoaderInternals {
    * @return true if the vector can expand, false if an overflow
    * event should occur
    */
-
   boolean canExpand(int delta);
 
   /**
@@ -88,7 +83,6 @@ interface LoaderInternals {
    * position, that should allow saving of the in-flight value (unless
    * that one value is larger than the maximum vector size.)
    */
-
   void overflowed();
 
   /**
@@ -96,7 +90,6 @@ interface LoaderInternals {
    *
    * @return the current write row index
    */
-
   int rowIndex();
 
   /**
@@ -104,7 +97,6 @@ interface LoaderInternals {
    *
    * @return the target row count
    */
-
   int targetRowCount();
 
   /**
@@ -112,10 +104,11 @@ interface LoaderInternals {
    * occurs when a batch has been started, before the batch overflows
    * or is harvested.
    *
-   * @return <tt>true if values can be written to vectors
+   * @return {@code true} if values can be written to vectors
    */
-
   boolean writeable();
 
   ColumnBuilder columnBuilder();
+
+  CustomErrorContext errorContext();
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/NullResultVectorCacheImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/NullResultVectorCacheImpl.java
index db6ab50..36a981b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/NullResultVectorCacheImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/NullResultVectorCacheImpl.java
@@ -40,7 +40,7 @@ public class NullResultVectorCacheImpl implements ResultVectorCache {
   public BufferAllocator allocator() { return allocator; }
 
   @Override
-  public ValueVector addOrGet(MaterializedField colSchema) {
+  public ValueVector vectorFor(MaterializedField colSchema) {
     return TypeHelper.getNewVector(colSchema, allocator, null);
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ProjectionFilter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ProjectionFilter.java
new file mode 100644
index 0000000..d2250ef
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ProjectionFilter.java
@@ -0,0 +1,241 @@
+/*
+ * 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.resultSet.impl;
+
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.physical.resultSet.project.Projections;
+import org.apache.drill.exec.physical.resultSet.project.RequestedTuple;
+import org.apache.drill.exec.physical.resultSet.project.RequestedTuple.TupleProjectionType;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Projection filter used when adding columns to the result set loader.
+ * Provides a variety of ways to filter columns: no filtering, filter
+ * by (parsed) projection list, or filter by projection list and
+ * provided schema. Enforces consistency of actual reader schema and
+ * projection list and/or provided schema.
+ * <p>
+ * Projection filters should not need to be extensible; filtering
+ * depends only on projection and provided schema.
+ */
+public interface ProjectionFilter {
+  Logger logger = LoggerFactory.getLogger(ProjectionFilter.class);
+
+  ProjectionFilter PROJECT_ALL = new ImplicitProjectionFilter(true);
+  ProjectionFilter PROJECT_NONE = new ImplicitProjectionFilter(false);
+
+  boolean isProjected(ColumnMetadata columnSchema);
+
+  ProjectionFilter mapProjection(boolean isColProjected, String colName);
+
+  boolean isEmpty();
+
+  public static ProjectionFilter filterFor(RequestedTuple tupleProj,
+      CustomErrorContext errorContext) {
+    if (tupleProj.type() == TupleProjectionType.ALL) {
+      return PROJECT_ALL;
+    } else {
+      return new DirectProjectionFilter(tupleProj, errorContext);
+    }
+  }
+
+  public static ProjectionFilter filterFor(RequestedTuple tupleProj,
+      TupleMetadata providedSchema, CustomErrorContext errorContext) {
+    if (providedSchema == null) {
+      return filterFor(tupleProj, errorContext);
+    }
+    return new CompoundProjectionFilter(
+        new DirectProjectionFilter(tupleProj, errorContext),
+        new SchemaProjectionFilter(providedSchema, errorContext));
+  }
+
+  /**
+   * Implied projection: either project all or project none. Never
+   * projects special columns (those marked as not being expanded in
+   * SELECT *).
+   */
+  public static class ImplicitProjectionFilter implements ProjectionFilter {
+    private final boolean projectAll;
+
+    public ImplicitProjectionFilter(boolean projectAll) {
+      this.projectAll = projectAll;
+    }
+
+    @Override
+    public boolean isProjected(ColumnMetadata columnSchema) {
+      return projectAll ? !Projections.excludeFromWildcard(columnSchema) : false;
+    }
+
+    @Override
+    public ProjectionFilter mapProjection(boolean isColProjected, String colName) {
+      return isColProjected ? this : PROJECT_NONE;
+    }
+
+    @Override
+    public boolean isEmpty() {
+      return !projectAll;
+    }
+  }
+
+  /**
+   * Projection filter based on the (parsed) projection list. Enforces that
+   * the reader column is consistent with the form of projection (map,
+   * array, or plain) in the projection list.
+   */
+  public static class DirectProjectionFilter implements ProjectionFilter {
+    private final RequestedTuple projectionSet;
+    private final CustomErrorContext errorContext;
+
+    public DirectProjectionFilter(RequestedTuple projectionSet, CustomErrorContext errorContext) {
+      this.projectionSet = projectionSet;
+      this.errorContext = errorContext;
+    }
+
+    @Override
+    public boolean isProjected(ColumnMetadata columnSchema) {
+      return projectionSet.enforceProjection(columnSchema, errorContext);
+    }
+
+    @Override
+    public ProjectionFilter mapProjection(boolean isColProjected, String colName) {
+      return isColProjected ?
+        filterFor(projectionSet.mapProjection(colName), errorContext) :
+        PROJECT_NONE;
+    }
+
+    @Override
+    public boolean isEmpty() {
+      return projectionSet.isEmpty();
+    }
+  }
+
+  /**
+   * Projection based on a provided schema. If the schema is strict, a reader column
+   * is projected only if that column appears in the provided schema. Non-strict
+   * schema allow additional reader columns.
+   * <p>
+   * If the column is found, enforces that the reader schema has the same type and
+   * mode as the provided column.
+   */
+  public static class SchemaProjectionFilter implements ProjectionFilter {
+    private final TupleMetadata providedSchema;
+    private final CustomErrorContext errorContext;
+    private final boolean isStrict;
+
+    public SchemaProjectionFilter(TupleMetadata providedSchema, CustomErrorContext errorContext) {
+      this(providedSchema,
+          providedSchema.booleanProperty(TupleMetadata.IS_STRICT_SCHEMA_PROP),
+          errorContext);
+    }
+
+    private SchemaProjectionFilter(TupleMetadata providedSchema, boolean isStrict, CustomErrorContext errorContext) {
+      this.providedSchema = providedSchema;
+      this.errorContext = errorContext;
+      this.isStrict = isStrict;
+    }
+
+    @Override
+    public boolean isProjected(ColumnMetadata columnSchema) {
+      ColumnMetadata providedCol = providedSchema.metadata(columnSchema.name());
+      if (providedCol == null) {
+        return !isStrict;
+      }
+      if (providedCol.type() != columnSchema.type() ||
+          providedCol.mode() != columnSchema.mode()) {
+        throw UserException.validationError()
+          .message("Reader and provided column type mismatch")
+          .addContext("Provided column", providedCol.columnString())
+          .addContext("Reader column", columnSchema.columnString())
+          .addContext(errorContext)
+          .build(logger);
+      }
+      return true;
+    }
+
+    @Override
+    public ProjectionFilter mapProjection(boolean isColProjected, String colName) {
+      if (!isColProjected) {
+        return PROJECT_NONE;
+      }
+      ColumnMetadata providedCol = providedSchema.metadata(colName);
+      if (providedCol == null) {
+        return PROJECT_ALL;
+      }
+      if (!providedCol.isMap()) {
+        throw UserException.validationError()
+          .message("Reader expected a map column, but the the provided column is not a map")
+          .addContext("Provided column", providedCol.columnString())
+          .addContext("Reader column", colName)
+          .addContext(errorContext)
+          .build(logger);
+      }
+      return new SchemaProjectionFilter(providedCol.tupleSchema(), isStrict, errorContext);
+    }
+
+    @Override
+    public boolean isEmpty() {
+       return providedSchema.isEmpty();
+    }
+  }
+
+  /**
+   * Compound filter for combining direct and provided schema projections.
+   */
+  public static class CompoundProjectionFilter implements ProjectionFilter {
+    private final ProjectionFilter filter1;
+    private final ProjectionFilter filter2;
+
+    public CompoundProjectionFilter(ProjectionFilter filter1, ProjectionFilter filter2) {
+      this.filter1 = filter1;
+      this.filter2 = filter2;
+    }
+
+    @Override
+    public boolean isProjected(ColumnMetadata columnSchema) {
+      return filter1.isProjected(columnSchema) && filter2.isProjected(columnSchema);
+    }
+
+    @Override
+    public ProjectionFilter mapProjection(boolean isColProjected, String colName) {
+      ProjectionFilter childFilter1 = filter1.mapProjection(isColProjected, colName);
+      ProjectionFilter childFilter2 = filter2.mapProjection(isColProjected, colName);
+      if (childFilter1 == PROJECT_ALL) {
+        return childFilter2;
+      }
+      if (childFilter1 == PROJECT_NONE) {
+        return childFilter1;
+      }
+      if (childFilter2 == PROJECT_ALL) {
+        return childFilter1;
+      }
+      if (childFilter2 == PROJECT_NONE) {
+        return childFilter2;
+      }
+      return new CompoundProjectionFilter(childFilter1, childFilter2);
+    }
+
+    @Override
+    public boolean isEmpty() {
+      return filter1.isEmpty() && filter2.isEmpty();
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/RepeatedListState.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/RepeatedListState.java
index f5b7101..9037cc9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/RepeatedListState.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/RepeatedListState.java
@@ -20,7 +20,6 @@ package org.apache.drill.exec.physical.resultSet.impl;
 import java.util.ArrayList;
 import java.util.Collection;
 
-import org.apache.drill.exec.physical.impl.scan.project.projSet.ProjectionSetFactory;
 import org.apache.drill.exec.physical.resultSet.ResultVectorCache;
 import org.apache.drill.exec.physical.resultSet.impl.ColumnState.BaseContainerColumnState;
 import org.apache.drill.exec.physical.resultSet.impl.SingleVectorState.OffsetVectorState;
@@ -42,13 +41,11 @@ import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
  * implements. At the vector level, we track the repeated list vector, but
  * only perform operations on its associated offset vector.
  */
-
 public class RepeatedListState extends ContainerState implements RepeatedListWriter.ArrayListener {
 
   /**
    * Repeated list column state.
    */
-
   public static class RepeatedListColumnState extends BaseContainerColumnState {
 
     private final RepeatedListState listState;
@@ -79,7 +76,6 @@ public class RepeatedListState extends ContainerState implements RepeatedListWri
    * offset vector. The child column state manages the repeated list content
    * (which may be complex: another repeated list, a map, a union, etc.)
    */
-
   public static class RepeatedListVectorState implements VectorState {
 
     private final ArrayWriter arrayWriter;
@@ -102,7 +98,6 @@ public class RepeatedListState extends ContainerState implements RepeatedListWri
      * @param childWriter child array writer for the inner dimension
      * of the repeated list
      */
-
     public void updateChildWriter(AbstractObjectWriter childWriter) {
       offsetsState.setChildWriter(childWriter.events());
     }
@@ -157,7 +152,7 @@ public class RepeatedListState extends ContainerState implements RepeatedListWri
 
   public RepeatedListState(LoaderInternals loader,
       ResultVectorCache vectorCache) {
-    super(loader, vectorCache, ProjectionSetFactory.projectAll());
+    super(loader, vectorCache, ProjectionFilter.PROJECT_ALL);
   }
 
   @Override
@@ -169,18 +164,15 @@ public class RepeatedListState extends ContainerState implements RepeatedListWri
   protected void addColumn(ColumnState colState) {
 
     // Remember the one and only child column.
-
     assert childState == null;
     childState = colState;
 
     // Add the new child schema to the existing repeated list
     // schema.
-
     ((RepeatedListColumnMetadata) parentColumn.schema()).childSchema(colState.schema());
 
     // Add the child vector to the existing repeated list
     // vector.
-
     final RepeatedListVectorState vectorState = (RepeatedListVectorState) parentColumn.vectorState();
     final RepeatedListVector listVector = vectorState.vector;
     listVector.setChildVector(childState.vector());
@@ -188,7 +180,6 @@ public class RepeatedListState extends ContainerState implements RepeatedListWri
     // The repeated list's offset vector state needs to know the offset
     // of the inner vector. Bind that information now that we have
     // an inner writer.
-
     vectorState.updateChildWriter(childState.writer());
   }
 
@@ -197,7 +188,6 @@ public class RepeatedListState extends ContainerState implements RepeatedListWri
 
     // Turn the one and only child into a list of children for
     // the general container mechanism.
-
     if (childState == null) {
       return new ArrayList<>();
     } else {
@@ -214,12 +204,10 @@ public class RepeatedListState extends ContainerState implements RepeatedListWri
    * and 2) given that background, the additional work of versioning is
    * not worth the effort.
    */
-
   @Override
   protected boolean isVersioned() { return false; }
 
   // Callback from the repeated list vector to add the child.
-
   @Override
   public AbstractObjectWriter setChild(ArrayWriter array,
       ColumnMetadata columnSchema) {
@@ -229,7 +217,6 @@ public class RepeatedListState extends ContainerState implements RepeatedListWri
   }
 
   // Callback from the repeated list vector to add the child.
-
   @Override
   public AbstractObjectWriter setChild(ArrayWriter array,
       MaterializedField field) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ResultSetCopierImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ResultSetCopierImpl.java
index bd7b6a0..9c596ff 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ResultSetCopierImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ResultSetCopierImpl.java
@@ -75,7 +75,7 @@ public class ResultSetCopierImpl implements ResultSetCopier {
   // Output state
 
   private final BufferAllocator allocator;
-  private final OptionBuilder writerOptions;
+  private final ResultSetOptionBuilder writerOptions;
   private ResultSetLoader resultSetWriter;
   private RowSetLoader rowWriter;
 
@@ -86,15 +86,15 @@ public class ResultSetCopierImpl implements ResultSetCopier {
   private CopyAll activeCopy;
 
   public ResultSetCopierImpl(BufferAllocator allocator, BatchAccessor inputBatch) {
-    this(allocator, inputBatch, new OptionBuilder());
+    this(allocator, inputBatch, new ResultSetOptionBuilder());
   }
 
   public ResultSetCopierImpl(BufferAllocator allocator, BatchAccessor inputBatch,
-      OptionBuilder outputOptions) {
+      ResultSetOptionBuilder outputOptions) {
     this.allocator = allocator;
     resultSetReader = new ResultSetReaderImpl(inputBatch);
     writerOptions = outputOptions;
-    writerOptions.setVectorCache(new ResultVectorCacheImpl(allocator));
+    writerOptions.vectorCache(new ResultVectorCacheImpl(allocator));
     state = State.START;
   }
 
@@ -194,7 +194,7 @@ public class ResultSetCopierImpl implements ResultSetCopier {
       resultSetWriter.close();
     }
     TupleMetadata schema = MetadataUtils.fromFields(resultSetReader.inputBatch().schema());
-    writerOptions.setSchema(schema);
+    writerOptions.readerSchema(schema);
     resultSetWriter = new ResultSetLoaderImpl(allocator, writerOptions.build());
     rowWriter = resultSetWriter.writer();
     currentSchemaVersion = resultSetReader.inputBatch().schemaVersion();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ResultSetLoaderImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ResultSetLoaderImpl.java
index c0695fa..cee582b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ResultSetLoaderImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ResultSetLoaderImpl.java
@@ -20,8 +20,6 @@ package org.apache.drill.exec.physical.resultSet.impl;
 import org.apache.drill.common.exceptions.CustomErrorContext;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.physical.impl.scan.project.projSet.ProjectionSetFactory;
-import org.apache.drill.exec.physical.resultSet.ProjectionSet;
 import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
 import org.apache.drill.exec.physical.resultSet.ResultVectorCache;
 import org.apache.drill.exec.physical.resultSet.RowSetLoader;
@@ -40,6 +38,7 @@ import org.slf4j.LoggerFactory;
  * @see {@link ResultSetLoader}
  */
 public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
+  protected static final Logger logger = LoggerFactory.getLogger(ResultSetLoaderImpl.class);
 
   /**
    * Read-only set of options for the result set loader.
@@ -48,7 +47,7 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
     protected final int vectorSizeLimit;
     protected final int rowCountLimit;
     protected final ResultVectorCache vectorCache;
-    protected final ProjectionSet projectionSet;
+    protected final ProjectionFilter projectionSet;
     protected final TupleMetadata schema;
     protected final long maxBatchSize;
 
@@ -60,23 +59,27 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
     public ResultSetOptions() {
       vectorSizeLimit = ValueVector.MAX_BUFFER_SIZE;
       rowCountLimit = DEFAULT_ROW_COUNT;
-      projectionSet = ProjectionSetFactory.projectAll();
+      projectionSet = ProjectionFilter.PROJECT_ALL;
       vectorCache = null;
       schema = null;
       maxBatchSize = -1;
       errorContext = null;
     }
 
-    public ResultSetOptions(OptionBuilder builder) {
+    public ResultSetOptions(ResultSetOptionBuilder builder) {
       vectorSizeLimit = builder.vectorSizeLimit;
       rowCountLimit = builder.rowCountLimit;
       vectorCache = builder.vectorCache;
-      schema = builder.schema;
+      schema = builder.readerSchema;
       maxBatchSize = builder.maxBatchSize;
       errorContext = builder.errorContext;
-      projectionSet = builder.projectionSet == null ?
-          ProjectionSetFactory.projectAll() :
-          builder.projectionSet;
+      if (builder.projectionFilter != null) {
+        projectionSet = builder.projectionFilter;
+      } else if (builder.projectionSet != null) {
+        projectionSet = ProjectionFilter.filterFor(builder.projectionSet, errorContext);
+      } else {
+        projectionSet = ProjectionFilter.PROJECT_ALL;
+      }
     }
 
     public void dump(HierarchicalFormatter format) {
@@ -155,8 +158,6 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
     CLOSED
   }
 
-  protected static final Logger logger = LoggerFactory.getLogger(ResultSetLoaderImpl.class);
-
   /**
    * Options provided to this loader.
    */
@@ -251,8 +252,6 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
    */
   protected int accumulatedBatchSize;
 
-  protected final ProjectionSet projectionSet;
-
   public ResultSetLoaderImpl(BufferAllocator allocator, ResultSetOptions options) {
     this.allocator = allocator;
     this.options = options;
@@ -260,10 +259,6 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
     writerIndex = new WriterIndexImpl(this);
     columnBuilder = new ColumnBuilder();
 
-    // Set the projections
-
-    projectionSet = options.projectionSet;
-
     // Determine the root vector cache
 
     ResultVectorCache vectorCache;
@@ -297,12 +292,15 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
     //
     // This accomplishes a result similar to the "legacy" readers
     // achieve by adding a dummy column.
-
-    if (projectionSet.isEmpty()) {
+    if (projectionSet().isEmpty()) {
       bumpVersion();
     }
   }
 
+  public ProjectionFilter projectionSet() {
+    return options.projectionSet;
+  }
+
   private void updateCardinality() {
     rootState.updateCardinality();
   }
@@ -325,14 +323,12 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
 
     activeSchemaVersion++;
     switch (state) {
-    case HARVESTED:
-    case START:
-    case LOOK_AHEAD:
-      harvestSchemaVersion = activeSchemaVersion;
-      break;
-    default:
-      break;
-
+      case HARVESTED:
+      case START:
+      case LOOK_AHEAD:
+        harvestSchemaVersion = activeSchemaVersion;
+        break;
+      default:
     }
     return activeSchemaVersion;
   }
@@ -343,27 +339,24 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
   @Override
   public int schemaVersion() {
     switch (state) {
-    case ACTIVE:
-    case IN_OVERFLOW:
-    case OVERFLOW:
-    case FULL_BATCH:
-
-      // Write in progress: use current writer schema
-
-      return activeSchemaVersion;
-    case HARVESTED:
-    case LOOK_AHEAD:
-    case START:
-
-      // Batch is published. Use harvest schema.
-
-      return harvestSchemaVersion;
-    default:
-
-      // Not really in a position to give a schema
-      // version.
-
-      throw new IllegalStateException("Unexpected state: " + state);
+      case ACTIVE:
+      case IN_OVERFLOW:
+      case OVERFLOW:
+      case FULL_BATCH:
+
+        // Write in progress: use current writer schema
+        return activeSchemaVersion;
+      case HARVESTED:
+      case LOOK_AHEAD:
+      case START:
+
+        // Batch is published. Use harvest schema.
+        return harvestSchemaVersion;
+      default:
+
+        // Not really in a position to give a schema
+        // version.
+        throw new IllegalStateException("Unexpected state: " + state);
     }
   }
 
@@ -381,45 +374,41 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
 
   public void startBatch(boolean schemaOnly) {
     switch (state) {
-    case HARVESTED:
-    case START:
-      logger.trace("Start batch");
-      accumulatedBatchSize = 0;
-      updateCardinality();
-      rootState.startBatch(schemaOnly);
-      checkInitialAllocation();
-
-      // The previous batch ended without overflow, so start
-      // a new batch, and reset the write index to 0.
-
-      writerIndex.reset();
-      rootWriter.startWrite();
-      break;
-
-    case LOOK_AHEAD:
-
-      // A row overflowed so keep the writer index at its current value
-      // as it points to the second row in the overflow batch. However,
-      // the last write position of each writer must be restored on
-      // a column-by-column basis, which is done by the visitor.
-
-      logger.trace("Start batch after overflow");
-      rootState.startBatch(schemaOnly);
-
-      // Note: no need to do anything with the writers; they were left
-      // pointing to the correct positions in the look-ahead batch.
-      // The above simply puts the look-ahead vectors back "under"
-      // the writers.
-
-      break;
-
-    default:
-      throw new IllegalStateException("Unexpected state: " + state);
+      case HARVESTED:
+      case START:
+        logger.trace("Start batch");
+        accumulatedBatchSize = 0;
+        updateCardinality();
+        rootState.startBatch(schemaOnly);
+        checkInitialAllocation();
+
+        // The previous batch ended without overflow, so start
+        // a new batch, and reset the write index to 0.
+        writerIndex.reset();
+        rootWriter.startWrite();
+        break;
+
+      case LOOK_AHEAD:
+
+        // A row overflowed so keep the writer index at its current value
+        // as it points to the second row in the overflow batch. However,
+        // the last write position of each writer must be restored on
+        // a column-by-column basis, which is done by the visitor.
+        logger.trace("Start batch after overflow");
+        rootState.startBatch(schemaOnly);
+
+        // Note: no need to do anything with the writers; they were left
+        // pointing to the correct positions in the look-ahead batch.
+        // The above simply puts the look-ahead vectors back "under"
+        // the writers.
+        break;
+
+      default:
+        throw new IllegalStateException("Unexpected state: " + state);
     }
 
     // Update the visible schema with any pending overflow batch
-    // updates.
-
+    // updates
     harvestSchemaVersion = activeSchemaVersion;
     pendingRowCount = 0;
     state = State.ACTIVE;
@@ -428,14 +417,14 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
   @Override
   public boolean hasRows() {
     switch (state) {
-    case ACTIVE:
-    case HARVESTED:
-    case FULL_BATCH:
-      return rootWriter.rowCount() > 0;
-    case LOOK_AHEAD:
-      return true;
-    default:
-      return false;
+      case ACTIVE:
+      case HARVESTED:
+      case FULL_BATCH:
+        return rootWriter.rowCount() > 0;
+      case LOOK_AHEAD:
+        return true;
+      default:
+        return false;
     }
   }
 
@@ -461,16 +450,15 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
    */
   protected void startRow() {
     switch (state) {
-    case ACTIVE:
-
-      // Update the visible schema with any pending overflow batch
-      // updates.
-
-      harvestSchemaVersion = activeSchemaVersion;
-      rootWriter.startRow();
-      break;
-    default:
-      throw new IllegalStateException("Unexpected state: " + state);
+      case ACTIVE:
+
+        // Update the visible schema with any pending overflow batch
+        // updates.
+        harvestSchemaVersion = activeSchemaVersion;
+        rootWriter.startRow();
+        break;
+      default:
+        throw new IllegalStateException("Unexpected state: " + state);
     }
   }
 
@@ -480,41 +468,37 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
    */
   protected void saveRow() {
     switch (state) {
-    case ACTIVE:
-      rootWriter.endArrayValue();
-      rootWriter.saveRow();
-      if (!writerIndex.next()) {
-        state = State.FULL_BATCH;
-      }
-
-      // No overflow row. Advertise the schema version to the client.
-
-      harvestSchemaVersion = activeSchemaVersion;
-      break;
-
-    case OVERFLOW:
-
-      // End the value of the look-ahead row in the look-ahead vectors.
-
-      rootWriter.endArrayValue();
-      rootWriter.saveRow();
-
-      // Advance the writer index relative to the look-ahead batch.
-
-      writerIndex.next();
-
-      // Stay in the overflow state. Doing so will cause the writer
-      // to report that it is full.
-      //
-      // Also, do not change the harvest schema version. We will
-      // expose to the downstream operators the schema in effect
-      // at the start of the row. Columns added within the row won't
-      // appear until the next batch.
-
-      break;
-
-    default:
-      throw new IllegalStateException("Unexpected state: " + state);
+      case ACTIVE:
+        rootWriter.endArrayValue();
+        rootWriter.saveRow();
+        if (!writerIndex.next()) {
+          state = State.FULL_BATCH;
+        }
+
+        // No overflow row. Advertise the schema version to the client.
+        harvestSchemaVersion = activeSchemaVersion;
+        break;
+
+      case OVERFLOW:
+
+        // End the value of the look-ahead row in the look-ahead vectors.
+        rootWriter.endArrayValue();
+        rootWriter.saveRow();
+
+        // Advance the writer index relative to the look-ahead batch.
+        writerIndex.next();
+
+        // Stay in the overflow state. Doing so will cause the writer
+        // to report that it is full.
+        //
+        // Also, do not change the harvest schema version. We will
+        // expose to the downstream operators the schema in effect
+        // at the start of the row. Columns added within the row won't
+        // appear until the next batch.
+        break;
+
+      default:
+        throw new IllegalStateException("Unexpected state: " + state);
     }
   }
 
@@ -525,13 +509,13 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
    */
   protected boolean isFull() {
     switch (state) {
-    case ACTIVE:
-      return !writerIndex.valid();
-    case OVERFLOW:
-    case FULL_BATCH:
-      return true;
-    default:
-      return false;
+      case ACTIVE:
+        return !writerIndex.valid();
+      case OVERFLOW:
+      case FULL_BATCH:
+        return true;
+      default:
+        return false;
     }
   }
 
@@ -553,13 +537,13 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
    */
   protected int rowCount() {
     switch (state) {
-    case ACTIVE:
-    case FULL_BATCH:
-      return writerIndex.size();
-    case OVERFLOW:
-      return pendingRowCount;
-    default:
-      return 0;
+      case ACTIVE:
+      case FULL_BATCH:
+        return writerIndex.size();
+      case OVERFLOW:
+        return pendingRowCount;
+      default:
+        return 0;
     }
   }
 
@@ -580,13 +564,11 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
   public int skipRows(int requestedCount) {
 
     // Can only skip rows when a batch is active.
-
     if (state != State.ACTIVE) {
       throw new IllegalStateException("No batch is active.");
     }
 
     // Skip as many rows as the vector limit allows.
-
     return writerIndex.skipRows(requestedCount);
   }
 
@@ -626,12 +608,10 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
     state = State.IN_OVERFLOW;
 
     // Preserve the number of rows in the now-complete batch.
-
     pendingRowCount = writerIndex.vectorIndex();
 
     // Roll-over will allocate new vectors. Update with the latest
     // array cardinality.
-
     updateCardinality();
 
     // Wrap up the completed rows into a batch. Sets
@@ -639,11 +619,9 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
     // it can be moved, but it is now past the recorded
     // end of the vectors (though, obviously, not past the
     // physical end.)
-
     rootWriter.preRollover();
 
     // Roll over vector values.
-
     accumulatedBatchSize = 0;
     rootState.rollover();
 
@@ -651,7 +629,6 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
     // surprisingly easy if we note that the current row is shifted to
     // the 0 position in the new vector, so we just shift all offsets
     // downward by the current row position at each repeat level.
-
     rootWriter.postRollover();
 
     // The writer index is reset back to 0. Because of the above roll-over
@@ -666,12 +643,10 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
     // the array values have been moved, offset vectors adjusted, the
     // element writer adjusted, so that v4 will be written to index 3
     // to produce (v1, v2, v3, v4, v5, ...) in the look-ahead vector.
-
     writerIndex.rollover();
     checkInitialAllocation();
 
     // Remember that overflow is in effect.
-
     state = State.OVERFLOW;
   }
 
@@ -687,17 +662,17 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
   public VectorContainer harvest() {
     int rowCount;
     switch (state) {
-    case ACTIVE:
-    case FULL_BATCH:
-      rowCount = harvestNormalBatch();
-      logger.trace("Harvesting {} rows", rowCount);
-      break;
-    case OVERFLOW:
-      rowCount = harvestOverflowBatch();
-      logger.trace("Harvesting {} rows after overflow", rowCount);
-      break;
-    default:
-      throw new IllegalStateException("Unexpected state: " + state);
+      case ACTIVE:
+      case FULL_BATCH:
+        rowCount = harvestNormalBatch();
+        logger.trace("Harvesting {} rows", rowCount);
+        break;
+      case OVERFLOW:
+        rowCount = harvestOverflowBatch();
+        logger.trace("Harvesting {} rows after overflow", rowCount);
+        break;
+      default:
+        throw new IllegalStateException("Unexpected state: " + state);
     }
 
     rootState.updateOutput(harvestSchemaVersion);
@@ -741,7 +716,6 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
 
     // Do not close the vector cache; the caller owns that and
     // will, presumably, reuse those vectors for another writer.
-
     state = State.CLOSED;
   }
 
@@ -827,5 +801,5 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
   public ColumnBuilder columnBuilder() { return columnBuilder; }
 
   @Override
-  public CustomErrorContext context() { return options.errorContext; }
+  public CustomErrorContext errorContext() { return options.errorContext; }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/OptionBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ResultSetOptionBuilder.java
similarity index 80%
rename from exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/OptionBuilder.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ResultSetOptionBuilder.java
index 564e5de..3a9c8cb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/OptionBuilder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ResultSetOptionBuilder.java
@@ -18,9 +18,9 @@
 package org.apache.drill.exec.physical.resultSet.impl;
 
 import org.apache.drill.common.exceptions.CustomErrorContext;
-import org.apache.drill.exec.physical.resultSet.ProjectionSet;
 import org.apache.drill.exec.physical.resultSet.ResultVectorCache;
 import org.apache.drill.exec.physical.resultSet.impl.ResultSetLoaderImpl.ResultSetOptions;
+import org.apache.drill.exec.physical.resultSet.project.RequestedTuple;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.vector.BaseValueVector;
 import org.apache.drill.exec.vector.ValueVector;
@@ -30,13 +30,13 @@ import org.apache.drill.exec.vector.ValueVector;
  * are provided for all options; use the default options for test code or
  * for clients that don't need special settings.
  */
-
-public class OptionBuilder {
+public class ResultSetOptionBuilder {
   protected int vectorSizeLimit;
   protected int rowCountLimit;
   protected ResultVectorCache vectorCache;
-  protected ProjectionSet projectionSet;
-  protected TupleMetadata schema;
+  protected RequestedTuple projectionSet;
+  protected ProjectionFilter projectionFilter;
+  protected TupleMetadata readerSchema;
   protected long maxBatchSize;
 
   /**
@@ -44,7 +44,7 @@ public class OptionBuilder {
    */
   protected CustomErrorContext errorContext;
 
-  public OptionBuilder() {
+  public ResultSetOptionBuilder() {
     // Start with the default option values.
     ResultSetOptions options = new ResultSetOptions();
     vectorSizeLimit = options.vectorSizeLimit;
@@ -61,14 +61,13 @@ public class OptionBuilder {
    * @param limit the row count limit
    * @return this builder
    */
-
-  public OptionBuilder setRowCountLimit(int limit) {
+  public ResultSetOptionBuilder rowCountLimit(int limit) {
     rowCountLimit = Math.max(1,
         Math.min(limit, ValueVector.MAX_ROW_COUNT));
     return this;
   }
 
-  public OptionBuilder setBatchSizeLimit(int bytes) {
+  public ResultSetOptionBuilder batchSizeLimit(int bytes) {
     maxBatchSize = bytes;
     return this;
   }
@@ -83,8 +82,7 @@ public class OptionBuilder {
    * each request. If a true cache is needed, the caller must provide one
    * here.
    */
-
-  public OptionBuilder setVectorCache(ResultVectorCache vectorCache) {
+  public ResultSetOptionBuilder vectorCache(ResultVectorCache vectorCache) {
     this.vectorCache = vectorCache;
     return this;
   }
@@ -101,21 +99,25 @@ public class OptionBuilder {
    * @param schema the initial schema for the loader
    * @return this builder
    */
-
-  public OptionBuilder setSchema(TupleMetadata schema) {
-    this.schema = schema;
+  public ResultSetOptionBuilder readerSchema(TupleMetadata readerSchema) {
+    this.readerSchema = readerSchema;
     return this;
   }
 
-  public OptionBuilder setProjection(ProjectionSet projSet) {
+  public ResultSetOptionBuilder projection(RequestedTuple projSet) {
     this.projectionSet = projSet;
     return this;
   }
 
+  public ResultSetOptionBuilder projectionFilter(ProjectionFilter projectionFilter) {
+    this.projectionFilter = projectionFilter;
+    return this;
+  }
+
   /**
    * Provides context for error messages.
    */
-  public OptionBuilder setContext(CustomErrorContext context) {
+  public ResultSetOptionBuilder errorContext(CustomErrorContext context) {
     this.errorContext = context;
     return this;
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ResultVectorCacheImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ResultVectorCacheImpl.java
index f3c9752..b5ee493 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ResultVectorCacheImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ResultVectorCacheImpl.java
@@ -168,7 +168,7 @@ public class ResultVectorCacheImpl implements ResultVectorCache {
   }
 
   @Override
-  public ValueVector addOrGet(MaterializedField colSchema) {
+  public ValueVector vectorFor(MaterializedField colSchema) {
     VectorState vs = vectors.get(colSchema.getName());
 
     // If the vector is found, and is of the right type, reuse it.
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/RowSetLoaderImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/RowSetLoaderImpl.java
index 7e88725..4058fa7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/RowSetLoaderImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/RowSetLoaderImpl.java
@@ -32,7 +32,6 @@ import org.apache.drill.exec.vector.accessor.writer.AbstractTupleWriter;
  * operations are actually delegated to the result set loader, which handles
  * the details of working with overflow rows.
  */
-
 public class RowSetLoaderImpl extends AbstractTupleWriter implements RowSetLoader {
 
   private final ResultSetLoaderImpl rsLoader;
@@ -77,17 +76,14 @@ public class RowSetLoaderImpl extends AbstractTupleWriter implements RowSetLoade
     if (rsLoader.isFull()) {
 
       // Full batch? Return false.
-
       return false;
     } else if (state == State.IN_ROW) {
 
       // Already in a row? Rewind the to start of the row.
-
       restartRow();
     } else {
 
       // Otherwise, advance to the next row.
-
       rsLoader.startRow();
     }
     return true;
@@ -109,6 +105,7 @@ public class RowSetLoaderImpl extends AbstractTupleWriter implements RowSetLoade
 
   @Override
   public ColumnMetadata schema() {
+
     // The top-level tuple (the data row) is not associated
     // with a parent column. By contrast, a map tuple is
     // associated with the column that defines the map.
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/TupleState.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/TupleState.java
index 6138283..1f05ceb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/TupleState.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/TupleState.java
@@ -21,7 +21,6 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 
-import org.apache.drill.exec.physical.resultSet.ProjectionSet;
 import org.apache.drill.exec.physical.resultSet.ResultVectorCache;
 import org.apache.drill.exec.physical.resultSet.impl.ColumnState.BaseContainerColumnState;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
@@ -256,7 +255,7 @@ public abstract class TupleState extends ContainerState
     private final VectorContainer outputContainer;
 
     public RowState(ResultSetLoaderImpl rsLoader, ResultVectorCache vectorCache) {
-      super(rsLoader, vectorCache, rsLoader.projectionSet);
+      super(rsLoader, vectorCache, rsLoader.projectionSet());
       writer = new RowSetLoaderImpl(rsLoader, schema);
       writer.bindListener(this);
       outputContainer = new VectorContainer(rsLoader.allocator());
@@ -310,7 +309,7 @@ public abstract class TupleState extends ContainerState
 
     public MapState(LoaderInternals events,
         ResultVectorCache vectorCache,
-        ProjectionSet projectionSet) {
+        ProjectionFilter projectionSet) {
       super(events, vectorCache, projectionSet);
     }
 
@@ -381,7 +380,7 @@ public abstract class TupleState extends ContainerState
 
     public SingleMapState(LoaderInternals events,
         ResultVectorCache vectorCache,
-        ProjectionSet projectionSet) {
+        ProjectionFilter projectionSet) {
       super(events, vectorCache, projectionSet);
     }
 
@@ -400,7 +399,7 @@ public abstract class TupleState extends ContainerState
 
     public MapArrayState(LoaderInternals events,
         ResultVectorCache vectorCache,
-        ProjectionSet projectionSet) {
+        ProjectionFilter projectionSet) {
       super(events, vectorCache, projectionSet);
     }
 
@@ -447,7 +446,7 @@ public abstract class TupleState extends ContainerState
 
   protected TupleState(LoaderInternals events,
       ResultVectorCache vectorCache,
-      ProjectionSet projectionSet) {
+      ProjectionFilter projectionSet) {
     super(events, vectorCache, projectionSet);
   }
 
@@ -609,7 +608,7 @@ public abstract class TupleState extends ContainerState
 
     public DictState(LoaderInternals events,
                     ResultVectorCache vectorCache,
-                    ProjectionSet projectionSet) {
+                    ProjectionFilter projectionSet) {
       super(events, vectorCache, projectionSet);
     }
 
@@ -637,7 +636,7 @@ public abstract class TupleState extends ContainerState
 
     public SingleDictState(LoaderInternals events,
                           ResultVectorCache vectorCache,
-                          ProjectionSet projectionSet) {
+                          ProjectionFilter projectionSet) {
       super(events, vectorCache, projectionSet);
     }
 
@@ -651,7 +650,7 @@ public abstract class TupleState extends ContainerState
 
     public DictArrayState(LoaderInternals events,
                          ResultVectorCache vectorCache,
-                         ProjectionSet projectionSet) {
+                         ProjectionFilter projectionSet) {
       super(events, vectorCache, projectionSet);
     }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/model/single/BaseWriterBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/model/single/BaseWriterBuilder.java
index c8a0f97..b3212de 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/model/single/BaseWriterBuilder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/model/single/BaseWriterBuilder.java
@@ -27,7 +27,6 @@ import org.apache.drill.exec.physical.resultSet.model.MetadataProvider;
 import org.apache.drill.exec.physical.resultSet.model.MetadataProvider.VectorDescrip;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.vector.ValueVector;
-import org.apache.drill.exec.vector.accessor.convert.ColumnConversionFactory;
 import org.apache.drill.exec.vector.accessor.writer.AbstractObjectWriter;
 import org.apache.drill.exec.vector.accessor.writer.ColumnWriterFactory;
 import org.apache.drill.exec.vector.accessor.writer.ListWriterImpl;
@@ -55,15 +54,8 @@ import org.apache.drill.exec.vector.complex.UnionVector;
  * variant (LIST, UNION) and tuple (MAP) columns, the tree grows
  * quite complex.
  */
-
 public abstract class BaseWriterBuilder {
 
-  private final ColumnConversionFactory conversionFactory;
-
-  protected BaseWriterBuilder(ColumnConversionFactory conversionFactory) {
-    this.conversionFactory = conversionFactory;
-  }
-
   protected List<AbstractObjectWriter> buildContainerChildren(VectorContainer container,
       MetadataProvider mdProvider) {
     final List<AbstractObjectWriter> writers = new ArrayList<>();
@@ -92,7 +84,7 @@ public abstract class BaseWriterBuilder {
         return buildList(vector, descrip);
 
       default:
-        return ColumnWriterFactory.buildColumnWriter(descrip.metadata, conversionFactory, vector);
+        return ColumnWriterFactory.buildColumnWriter(descrip.metadata, vector);
     }
   }
 
@@ -179,7 +171,6 @@ public abstract class BaseWriterBuilder {
 
       // If the list holds a union, then the list and union are collapsed
       // together in the metadata layer.
-
       dataMetadata = descrip;
     } else {
       dataMetadata = new VectorDescrip(descrip.childProvider(), 0, dataVector.getField());
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/ImpliedTupleRequest.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/ImpliedTupleRequest.java
index 3b76d7e..09fc512 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/ImpliedTupleRequest.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/ImpliedTupleRequest.java
@@ -20,6 +20,9 @@ package org.apache.drill.exec.physical.resultSet.project;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+
 /**
  * Represents a wildcard: SELECT * when used at the root tuple.
  * When used with maps, means selection of all map columns, either
@@ -64,6 +67,17 @@ public class ImpliedTupleRequest implements RequestedTuple {
   }
 
   @Override
+  public boolean isProjected(ColumnMetadata columnSchema) {
+    return allProjected ? !Projections.excludeFromWildcard(columnSchema) : false;
+  }
+
+  @Override
+  public boolean enforceProjection(ColumnMetadata columnSchema,
+      CustomErrorContext errorContext) {
+    return isProjected(columnSchema);
+  }
+
+  @Override
   public String toString() {
     StringBuilder buf = new StringBuilder()
         .append("{");
@@ -72,4 +86,7 @@ public class ImpliedTupleRequest implements RequestedTuple {
     }
     return buf.append("}").toString();
   }
+
+  @Override
+  public boolean isEmpty() { return !allProjected; }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectionChecker.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/ProjectionChecker.java
similarity index 96%
rename from exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectionChecker.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/ProjectionChecker.java
index 8757167..3769f72 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectionChecker.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/ProjectionChecker.java
@@ -15,14 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.physical.impl.scan.project.projSet;
+package org.apache.drill.exec.physical.resultSet.project;
 
 import org.apache.drill.common.exceptions.CustomErrorContext;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.common.types.Types;
-import org.apache.drill.exec.physical.resultSet.project.RequestedColumn;
-import org.apache.drill.exec.physical.resultSet.project.RequestedTuple;
 import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/Projections.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/Projections.java
index a14a68d..8bf991f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/Projections.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/Projections.java
@@ -23,6 +23,7 @@ import java.util.List;
 import org.apache.drill.common.expression.PathSegment;
 import org.apache.drill.common.expression.PathSegment.ArraySegment;
 import org.apache.drill.common.expression.PathSegment.NameSegment;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.apache.drill.common.expression.SchemaPath;
 
 /**
@@ -156,4 +157,13 @@ public class Projections {
     }
     return new RequestedTupleImpl(projList);
   }
+
+  /**
+   * Reports whether the column is a special column which should not be
+   * expanded in a wildcard. Used for specialized columns in readers
+   * such as the Log format.
+   */
+  public static boolean excludeFromWildcard(ColumnMetadata col) {
+    return col.booleanProperty(ColumnMetadata.EXCLUDE_FROM_WILDCARD);
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/RequestedTuple.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/RequestedTuple.java
index 1279299..8d621a7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/RequestedTuple.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/RequestedTuple.java
@@ -19,6 +19,9 @@ package org.apache.drill.exec.physical.resultSet.project;
 
 import java.util.List;
 
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+
 /**
  * Represents the set of columns projected for a tuple (row or map.)
  * Each column may have structure: a set of referenced names or
@@ -58,7 +61,20 @@ public interface RequestedTuple {
   TupleProjectionType type();
   RequestedColumn get(String colName);
   boolean isProjected(String colName);
+  boolean isProjected(ColumnMetadata columnSchema);
+  boolean enforceProjection(ColumnMetadata columnSchema, CustomErrorContext errorContext);
   RequestedTuple mapProjection(String colName);
   List<RequestedColumn> projections();
   void buildName(StringBuilder buf);
+
+  /**
+   * Report if the projection is empty as occurs in
+   * {@code SELECT COUNT(*) FROM ...}. This is <i>not</i> the
+   * same as asking if this tuple is unprojected, as that concept
+   * does not apply to tuples, only to the column that contains the
+   * tuple.
+   *
+   * @return {@code true} if the projection set is empty
+   */
+  boolean isEmpty();
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/RequestedTupleImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/RequestedTupleImpl.java
index dc0c8e5..c871e86 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/RequestedTupleImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/RequestedTupleImpl.java
@@ -19,7 +19,9 @@ package org.apache.drill.exec.physical.resultSet.project;
 
 import java.util.List;
 
+import org.apache.drill.common.exceptions.CustomErrorContext;
 import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.apache.drill.exec.record.metadata.TupleNameSpace;
 
 /**
@@ -164,6 +166,27 @@ public class RequestedTupleImpl implements RequestedTuple {
   }
 
   @Override
+  public boolean isProjected(ColumnMetadata columnSchema) {
+    if (!isProjected(columnSchema.name())) {
+      return false;
+    }
+    return projectionType == TupleProjectionType.ALL ?
+        !Projections.excludeFromWildcard(columnSchema) : true;
+  }
+
+  public boolean enforceProjection(ColumnMetadata columnSchema, CustomErrorContext errorContext) {
+    if (projectionType == TupleProjectionType.ALL) {
+      return true;
+    }
+    RequestedColumn reqCol = get(columnSchema.name());
+    if (reqCol == null) {
+      return false;
+    }
+    ProjectionChecker.validateProjection(reqCol, columnSchema, errorContext);
+    return true;
+  }
+
+  @Override
   public RequestedTuple mapProjection(String colName) {
     switch (projectionType) {
       case ALL:
@@ -190,4 +213,7 @@ public class RequestedTupleImpl implements RequestedTuple {
     }
     return buf.append("}").toString();
   }
+
+  @Override
+  public boolean isEmpty() { return projections().isEmpty(); }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/DirectRowSet.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/DirectRowSet.java
index 67dd7e5..8574345 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/DirectRowSet.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/DirectRowSet.java
@@ -31,7 +31,6 @@ import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.record.VectorAccessible;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.selection.SelectionVector2;
-import org.apache.drill.exec.vector.accessor.convert.ColumnConversionFactory;
 import org.apache.drill.exec.physical.rowSet.RowSet.ExtendableRowSet;
 import org.apache.drill.exec.physical.rowSet.RowSetWriterImpl.WriterIndexImpl;
 
@@ -42,7 +41,6 @@ import java.util.Set;
  * Implementation of a single row set with no indirection (selection)
  * vector.
  */
-
 public class DirectRowSet extends AbstractSingleRowSet implements ExtendableRowSet {
 
   /**
@@ -52,10 +50,6 @@ public class DirectRowSet extends AbstractSingleRowSet implements ExtendableRowS
 
   public static class RowSetWriterBuilder extends BaseWriterBuilder {
 
-    public RowSetWriterBuilder(ColumnConversionFactory conversionFactory) {
-      super(conversionFactory);
-    }
-
     public RowSetWriter buildWriter(DirectRowSet rowSet) {
       WriterIndexImpl index = new WriterIndexImpl();
       TupleMetadata schema = rowSet.schema();
@@ -109,15 +103,11 @@ public class DirectRowSet extends AbstractSingleRowSet implements ExtendableRowS
 
   @Override
   public RowSetWriter writer(int initialRowCount) {
-    return writer(initialRowCount, null);
-  }
-
-  public RowSetWriter writer(int initialRowCount, ColumnConversionFactory conversionFactory) {
     if (container().hasRecordCount()) {
       throw new IllegalStateException("Row set already contains data");
     }
     allocate(initialRowCount);
-    return new RowSetWriterBuilder(conversionFactory).buildWriter(this);
+    return new RowSetWriterBuilder().buildWriter(this);
   }
 
   @Override
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/RowSetBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/RowSetBuilder.java
index 03dcb5c..2adb0df 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/RowSetBuilder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/RowSetBuilder.java
@@ -24,7 +24,6 @@ import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.metadata.MetadataUtils;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
-import org.apache.drill.exec.vector.accessor.convert.ColumnConversionFactory;
 import org.apache.drill.exec.physical.rowSet.RowSet.SingleRowSet;
 
 /**
@@ -34,10 +33,10 @@ import org.apache.drill.exec.physical.rowSet.RowSet.SingleRowSet;
 
 public final class RowSetBuilder {
 
-  private DirectRowSet rowSet;
-  private RowSetWriter writer;
+  private final DirectRowSet rowSet;
+  private final RowSetWriter writer;
   private boolean withSv2;
-  private Set<Integer> skipIndices = new HashSet<>();
+  private final Set<Integer> skipIndices = new HashSet<>();
 
   /**
    * Creates a {@link RowSetBuilder}. Since {@link BatchSchema} does not handle complex types well, this has been deprecated in favor of the other constructors.
@@ -48,26 +47,16 @@ public final class RowSetBuilder {
    */
   @Deprecated
   public RowSetBuilder(BufferAllocator allocator, BatchSchema schema) {
-    this(allocator, MetadataUtils.fromFields(schema), DirectRowSet.INITIAL_ROW_COUNT, null);
+    this(allocator, MetadataUtils.fromFields(schema), DirectRowSet.INITIAL_ROW_COUNT);
   }
 
   public RowSetBuilder(BufferAllocator allocator, TupleMetadata schema) {
-    this(allocator, schema, DirectRowSet.INITIAL_ROW_COUNT, null);
-  }
-
-  public RowSetBuilder(BufferAllocator allocator, TupleMetadata schema,
-      ColumnConversionFactory conversionFactory) {
-    this(allocator, schema, DirectRowSet.INITIAL_ROW_COUNT, conversionFactory);
+    this(allocator, schema, DirectRowSet.INITIAL_ROW_COUNT);
   }
 
   public RowSetBuilder(BufferAllocator allocator, TupleMetadata schema, int capacity) {
-    this(allocator, schema, capacity, null);
-  }
-
-  public RowSetBuilder(BufferAllocator allocator, TupleMetadata schema,
-      int capacity, ColumnConversionFactory conversionFactory) {
     rowSet = DirectRowSet.fromSchema(allocator, schema);
-    writer = rowSet.writer(capacity, conversionFactory);
+    writer = rowSet.writer(capacity);
   }
 
   public static RowSet emptyBatch(BufferAllocator allocator, TupleMetadata schema) {
@@ -89,7 +78,6 @@ public final class RowSetBuilder {
    *                               becomes full. This method is designed to be used in tests where we will
    *                               seldom create a full vector of data.
    */
-
   public RowSetBuilder addRow(Object... values) {
     writer.addRow(values);
     return this;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/RowSetWriterImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/RowSetWriterImpl.java
index 5854044..fd12c19 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/RowSetWriterImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/RowSetWriterImpl.java
@@ -33,19 +33,17 @@ import org.apache.drill.exec.physical.rowSet.RowSet.SingleRowSet;
  * empty, direct, single row sets. Rewriting is not allowed, nor is writing
  * to a hyper row set.
  */
-
 public class RowSetWriterImpl extends AbstractTupleWriter implements RowSetWriter {
 
   /**
    * Writer index that points to each row in the row set. The index starts at
    * the 0th row and advances one row on each increment. This allows writers to
    * start positioned at the first row. Writes happen in the current row.
-   * Calling <tt>next()</tt> advances to the next position, effectively saving
+   * Calling {@code next()} advances to the next position, effectively saving
    * the current row. The most recent row can be abandoned easily simply by not
-   * calling <tt>next()</tt>. This means that the number of completed rows is
+   * calling {@code next()}. This means that the number of completed rows is
    * the same as the row index.
    */
-
   static class WriterIndexImpl implements ColumnWriterIndex {
 
     public enum State { OK, VECTOR_OVERFLOW, END_OF_BATCH }
@@ -142,7 +140,6 @@ public class RowSetWriterImpl extends AbstractTupleWriter implements RowSetWrite
 
     // For convenience, start a new row after each save.
     // The last (unused) row is abandoned when the batch is full.
-
     if (writerIndex.next()) {
       startRow();
     }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/SchemaTreeProvider.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/SchemaTreeProvider.java
index dd00537..d981768 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/SchemaTreeProvider.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/SchemaTreeProvider.java
@@ -124,6 +124,7 @@ public class SchemaTreeProvider implements AutoCloseable {
     final SchemaConfig schemaConfig = SchemaConfig.newBuilder(schemaUser, provider).build();
     return createFullRootSchema(schemaConfig);
   }
+
   /**
    * Create and return a Full SchemaTree with given <i>schemaConfig</i>.
    * @param schemaConfig
@@ -150,7 +151,6 @@ public class SchemaTreeProvider implements AutoCloseable {
           .addContext(contextString)
           .build(logger);
     }
-
   }
 
   @Override
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroBatchReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroBatchReader.java
index 6213015..c22d453 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroBatchReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroBatchReader.java
@@ -23,6 +23,7 @@ import org.apache.avro.generic.GenericDatumReader;
 import org.apache.avro.generic.GenericRecord;
 import org.apache.avro.mapred.FsInput;
 import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.physical.impl.scan.convert.StandardConversions;
 import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
 import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
 import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
@@ -42,7 +43,6 @@ import java.util.List;
 import java.util.stream.IntStream;
 
 public class AvroBatchReader implements ManagedReader<FileScanFramework.FileSchemaNegotiator> {
-
   private static final Logger logger = LoggerFactory.getLogger(AvroBatchReader.class);
 
   private Path filePath;
@@ -51,7 +51,7 @@ public class AvroBatchReader implements ManagedReader<FileScanFramework.FileSche
   private ResultSetLoader loader;
   private List<ColumnConverter> converters;
   // re-use container instance
-  private GenericRecord record = null;
+  private GenericRecord record;
 
   @Override
   public boolean open(FileScanFramework.FileSchemaNegotiator negotiator) {
@@ -69,11 +69,15 @@ public class AvroBatchReader implements ManagedReader<FileScanFramework.FileSche
       negotiator.userName(), negotiator.context().getFragmentContext().getQueryUserName());
 
     logger.debug("Avro file schema: {}", reader.getSchema());
-    TupleMetadata schema = AvroSchemaUtil.convert(reader.getSchema());
-    logger.debug("Avro file converted schema: {}", schema);
-    negotiator.setTableSchema(schema, true);
+    TupleMetadata readerSchema = AvroSchemaUtil.convert(reader.getSchema());
+    logger.debug("Avro file converted schema: {}", readerSchema);
+    TupleMetadata providedSchema = negotiator.providedSchema();
+    TupleMetadata tableSchema = StandardConversions.mergeSchemas(providedSchema, readerSchema);
+    logger.debug("Avro file table schema: {}", tableSchema);
+    negotiator.tableSchema(tableSchema, true);
     loader = negotiator.build();
-    converters = ColumnConvertersUtil.initConverters(schema, loader.writer());
+    ColumnConverterFactory factory = new ColumnConverterFactory(providedSchema);
+    converters = factory.initConverters(providedSchema, readerSchema, loader.writer());
 
     return true;
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroFormatPlugin.java
index 952fb1e..7e7ad56 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroFormatPlugin.java
@@ -64,7 +64,7 @@ public class AvroFormatPlugin extends EasyFormatPlugin<AvroFormatConfig> {
     FileScanFramework.FileScanBuilder builder = new FileScanFramework.FileScanBuilder();
     builder.setReaderFactory(new AvroReaderFactory());
     initScanBuilder(builder, scan);
-    builder.setNullType(Types.optional(TypeProtos.MinorType.VARCHAR));
+    builder.nullType(Types.optional(TypeProtos.MinorType.VARCHAR));
     return builder;
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/ColumnConverter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/ColumnConverter.java
index 1a63f7b..ee856de 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/ColumnConverter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/ColumnConverter.java
@@ -17,30 +17,18 @@
  */
 package org.apache.drill.exec.store.avro;
 
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Consumer;
+import java.util.stream.IntStream;
+
 import org.apache.avro.generic.GenericArray;
-import org.apache.avro.generic.GenericFixed;
 import org.apache.avro.generic.GenericRecord;
-import org.apache.avro.util.Utf8;
-import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.vector.accessor.ArrayWriter;
 import org.apache.drill.exec.vector.accessor.DictWriter;
-import org.apache.drill.exec.vector.accessor.ScalarWriter;
 import org.apache.drill.exec.vector.accessor.TupleWriter;
-import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
-import org.joda.time.DateTimeConstants;
-import org.joda.time.Period;
-
-import java.math.BigDecimal;
-import java.math.BigInteger;
-import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
-import java.nio.IntBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.function.Consumer;
-import java.util.stream.IntStream;
 
 /**
  * Converts and sets given value into the specific column writer.
@@ -76,83 +64,6 @@ public interface ColumnConverter {
       this.valueConverter = valueConverter;
     }
 
-    public static ScalarColumnConverter init(ScalarWriter writer) {
-      ColumnMetadata columnMetadata = writer.schema();
-      switch (columnMetadata.type()) {
-        case VARCHAR:
-          return new ScalarColumnConverter(value -> {
-            byte[] binary;
-            int length;
-            if (value instanceof Utf8) {
-              Utf8 utf8 = (Utf8) value;
-              binary = utf8.getBytes();
-              length = utf8.getByteLength();
-            } else {
-              binary = value.toString().getBytes(Charsets.UTF_8);
-              length = binary.length;
-            }
-            writer.setBytes(binary, length);
-          });
-        case VARBINARY:
-          return new ScalarColumnConverter(value -> {
-            if (value instanceof ByteBuffer) {
-              ByteBuffer buf = (ByteBuffer) value;
-              writer.setBytes(buf.array(), buf.remaining());
-            } else {
-              byte[] bytes = ((GenericFixed) value).bytes();
-              writer.setBytes(bytes, bytes.length);
-            }
-          });
-        case VARDECIMAL:
-          return new ScalarColumnConverter(value -> {
-            BigInteger bigInteger;
-            if (value instanceof ByteBuffer) {
-              ByteBuffer decBuf = (ByteBuffer) value;
-              bigInteger = new BigInteger(decBuf.array());
-            } else {
-              GenericFixed genericFixed = (GenericFixed) value;
-              bigInteger = new BigInteger(genericFixed.bytes());
-            }
-            BigDecimal decimalValue = new BigDecimal(bigInteger, writer.schema().scale());
-            writer.setDecimal(decimalValue);
-          });
-        case TIMESTAMP:
-          return new ScalarColumnConverter(value -> {
-            String avroLogicalType = writer.schema().property(AvroSchemaUtil.AVRO_LOGICAL_TYPE_PROPERTY);
-            if (AvroSchemaUtil.TIMESTAMP_MILLIS_LOGICAL_TYPE.equals(avroLogicalType)) {
-              writer.setLong((long) value);
-            } else {
-              writer.setLong((long) value / 1000);
-            }
-          });
-        case DATE:
-          return new ScalarColumnConverter(value -> writer.setLong((int) value * (long) DateTimeConstants.MILLIS_PER_DAY));
-        case TIME:
-          return new ScalarColumnConverter(value -> {
-            if (value instanceof Long) {
-              writer.setInt((int) ((long) value / 1000));
-            } else {
-              writer.setInt((int) value);
-            }
-          });
-        case INTERVAL:
-          return new ScalarColumnConverter(value -> {
-            GenericFixed genericFixed = (GenericFixed) value;
-            IntBuffer intBuf = ByteBuffer.wrap(genericFixed.bytes())
-              .order(ByteOrder.LITTLE_ENDIAN)
-              .asIntBuffer();
-
-            Period period = Period.months(intBuf.get(0))
-              .withDays(intBuf.get(1)
-              ).withMillis(intBuf.get(2));
-
-            writer.setPeriod(period);
-          });
-        default:
-          return new ScalarColumnConverter(writer::setObject);
-      }
-    }
-
     @Override
     public void convert(Object value) {
       if (value == null) {
@@ -198,10 +109,16 @@ public interface ColumnConverter {
    */
   class MapColumnConverter implements ColumnConverter {
 
+    private final ColumnConverterFactory factory;
+    private final TupleMetadata providedSchema;
     private final TupleWriter tupleWriter;
     private final List<ColumnConverter> converters;
 
-    public MapColumnConverter(TupleWriter tupleWriter, List<ColumnConverter> converters) {
+    public MapColumnConverter(ColumnConverterFactory factory,
+        TupleMetadata providedSchema,
+        TupleWriter tupleWriter, List<ColumnConverter> converters) {
+      this.factory = factory;
+      this.providedSchema = providedSchema;
       this.tupleWriter = tupleWriter;
       this.converters = new ArrayList<>(converters);
     }
@@ -215,13 +132,7 @@ public interface ColumnConverter {
       GenericRecord genericRecord = (GenericRecord) value;
 
       if (converters.isEmpty()) {
-        // fill in tuple schema for cases when it contains recursive named record types
-        TupleMetadata metadata = AvroSchemaUtil.convert(genericRecord.getSchema());
-        metadata.toMetadataList().forEach(tupleWriter::addColumn);
-
-        IntStream.range(0, metadata.size())
-          .mapToObj(i -> ColumnConvertersUtil.getConverter(metadata.metadata(i), tupleWriter.column(i)))
-          .forEach(converters::add);
+        factory.buildMapMembers(genericRecord, providedSchema, tupleWriter, converters);
       }
 
       IntStream.range(0, converters.size())
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/ColumnConverterFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/ColumnConverterFactory.java
new file mode 100644
index 0000000..3c42227
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/ColumnConverterFactory.java
@@ -0,0 +1,257 @@
+/*
+ * 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.avro;
+
+import org.apache.avro.generic.GenericFixed;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.util.Utf8;
+import org.apache.drill.exec.physical.impl.scan.convert.StandardConversions;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.store.avro.ColumnConverter.ArrayColumnConverter;
+import org.apache.drill.exec.store.avro.ColumnConverter.DictColumnConverter;
+import org.apache.drill.exec.store.avro.ColumnConverter.DummyColumnConverter;
+import org.apache.drill.exec.store.avro.ColumnConverter.MapColumnConverter;
+import org.apache.drill.exec.store.avro.ColumnConverter.ScalarColumnConverter;
+import org.apache.drill.exec.vector.accessor.ArrayWriter;
+import org.apache.drill.exec.vector.accessor.DictWriter;
+import org.apache.drill.exec.vector.accessor.ObjectWriter;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.drill.exec.vector.accessor.ValueWriter;
+import org.apache.drill.exec.vector.complex.DictVector;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.joda.time.DateTimeConstants;
+import org.joda.time.Period;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.nio.IntBuffer;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+public class ColumnConverterFactory {
+
+  private final StandardConversions standardConversions;
+
+  public ColumnConverterFactory(TupleMetadata providedSchema) {
+    if (providedSchema == null) {
+      standardConversions = null;
+    } else {
+      standardConversions = new StandardConversions(providedSchema.properties());
+    }
+  }
+
+  /**
+   * Based on given converted Avro schema and current row writer generates list of
+   * column converters based on column type.
+   *
+   * @param readerSchema converted Avro schema
+   * @param rowWriter current row writer
+   * @return list of column converters
+   */
+  public List<ColumnConverter> initConverters(TupleMetadata providedSchema,
+      TupleMetadata readerSchema, RowSetLoader rowWriter) {
+    return IntStream.range(0, readerSchema.size())
+      .mapToObj(i -> getConverter(providedSchema, readerSchema.metadata(i), rowWriter.column(i)))
+      .collect(Collectors.toList());
+  }
+
+  /**
+   * Based on column type, creates corresponding column converter
+   * which holds conversion logic and appropriate writer to set converted data into.
+   * For columns which are not projected, {@link DummyColumnConverter} is used.
+   *
+   * @param readerSchema column metadata
+   * @param writer column writer
+   * @return column converter
+   */
+  public ColumnConverter getConverter(TupleMetadata providedSchema,
+      ColumnMetadata readerSchema, ObjectWriter writer) {
+    if (!writer.isProjected()) {
+      return DummyColumnConverter.INSTANCE;
+    }
+
+    if (readerSchema.isArray()) {
+      return getArrayConverter(providedSchema,
+          readerSchema, writer.array());
+    }
+
+    if (readerSchema.isMap()) {
+      return getMapConverter(
+          providedChildSchema(providedSchema, readerSchema),
+          readerSchema.tupleSchema(), writer.tuple());
+    }
+
+    if (readerSchema.isDict()) {
+      return getDictConverter(
+          providedChildSchema(providedSchema, readerSchema),
+          readerSchema.tupleSchema(), writer.dict());
+    }
+
+    return getScalarConverter(readerSchema, writer.scalar());
+  }
+
+  private TupleMetadata providedChildSchema(TupleMetadata providedSchema,
+      ColumnMetadata readerSchema) {
+    return providedSchema == null ? null :
+      providedSchema.metadata(readerSchema.name()).tupleSchema();
+  }
+
+  private ColumnConverter getArrayConverter(TupleMetadata providedSchema,
+      ColumnMetadata readerSchema, ArrayWriter arrayWriter) {
+    ObjectWriter valueWriter = arrayWriter.entry();
+    ColumnConverter valueConverter;
+    if (readerSchema.isMap()) {
+      valueConverter = getMapConverter(providedSchema,
+          readerSchema.tupleSchema(), valueWriter.tuple());
+    } else if (readerSchema.isDict()) {
+      valueConverter = getDictConverter(providedSchema,
+          readerSchema.tupleSchema(), valueWriter.dict());
+    } else if (readerSchema.isMultiList()) {
+      valueConverter = getConverter(null, readerSchema.childSchema(), valueWriter);
+    } else {
+      valueConverter = getScalarConverter(readerSchema, valueWriter.scalar());
+    }
+    return new ArrayColumnConverter(arrayWriter, valueConverter);
+  }
+
+  private ColumnConverter getMapConverter(TupleMetadata providedSchema,
+      TupleMetadata readerSchema, TupleWriter tupleWriter) {
+    List<ColumnConverter> converters = IntStream.range(0, readerSchema.size())
+      .mapToObj(i -> getConverter(providedSchema, readerSchema.metadata(i), tupleWriter.column(i)))
+      .collect(Collectors.toList());
+    return new MapColumnConverter(this, providedSchema, tupleWriter, converters);
+  }
+
+  private ColumnConverter getDictConverter(TupleMetadata providedSchema,
+      TupleMetadata readerSchema, DictWriter dictWriter) {
+    ColumnConverter keyConverter = getScalarConverter(
+        readerSchema.metadata(DictVector.FIELD_KEY_NAME), dictWriter.keyWriter());
+    ColumnConverter valueConverter = getConverter(providedSchema,
+        readerSchema.metadata(DictVector.FIELD_VALUE_NAME), dictWriter.valueWriter());
+    return new DictColumnConverter(dictWriter, keyConverter, valueConverter);
+  }
+
+  private ColumnConverter getScalarConverter(ColumnMetadata readerSchema, ScalarWriter scalarWriter) {
+    ValueWriter valueWriter;
+    if (standardConversions == null) {
+      valueWriter = scalarWriter;
+    } else {
+      valueWriter = standardConversions.converter(scalarWriter, readerSchema);
+    }
+    return buildScalar(readerSchema, valueWriter);
+  }
+
+  public  ScalarColumnConverter buildScalar(ColumnMetadata readerSchema, ValueWriter writer) {
+    switch (readerSchema.type()) {
+      case VARCHAR:
+        return new ScalarColumnConverter(value -> {
+          byte[] binary;
+          int length;
+          if (value instanceof Utf8) {
+            Utf8 utf8 = (Utf8) value;
+            binary = utf8.getBytes();
+            length = utf8.getByteLength();
+          } else {
+            binary = value.toString().getBytes(Charsets.UTF_8);
+            length = binary.length;
+          }
+          writer.setBytes(binary, length);
+        });
+      case VARBINARY:
+        return new ScalarColumnConverter(value -> {
+          if (value instanceof ByteBuffer) {
+            ByteBuffer buf = (ByteBuffer) value;
+            writer.setBytes(buf.array(), buf.remaining());
+          } else {
+            byte[] bytes = ((GenericFixed) value).bytes();
+            writer.setBytes(bytes, bytes.length);
+          }
+        });
+      case VARDECIMAL:
+        return new ScalarColumnConverter(value -> {
+          BigInteger bigInteger;
+          if (value instanceof ByteBuffer) {
+            ByteBuffer decBuf = (ByteBuffer) value;
+            bigInteger = new BigInteger(decBuf.array());
+          } else {
+            GenericFixed genericFixed = (GenericFixed) value;
+            bigInteger = new BigInteger(genericFixed.bytes());
+          }
+          BigDecimal decimalValue = new BigDecimal(bigInteger, readerSchema.scale());
+          writer.setDecimal(decimalValue);
+        });
+      case TIMESTAMP:
+        return new ScalarColumnConverter(value -> {
+          String avroLogicalType = readerSchema.property(AvroSchemaUtil.AVRO_LOGICAL_TYPE_PROPERTY);
+          if (AvroSchemaUtil.TIMESTAMP_MILLIS_LOGICAL_TYPE.equals(avroLogicalType)) {
+            writer.setLong((long) value);
+          } else {
+            writer.setLong((long) value / 1000);
+          }
+        });
+      case DATE:
+        return new ScalarColumnConverter(value -> writer.setLong((int) value * (long) DateTimeConstants.MILLIS_PER_DAY));
+      case TIME:
+        return new ScalarColumnConverter(value -> {
+          if (value instanceof Long) {
+            writer.setInt((int) ((long) value / 1000));
+          } else {
+            writer.setInt((int) value);
+          }
+        });
+      case INTERVAL:
+        return new ScalarColumnConverter(value -> {
+          GenericFixed genericFixed = (GenericFixed) value;
+          IntBuffer intBuf = ByteBuffer.wrap(genericFixed.bytes())
+            .order(ByteOrder.LITTLE_ENDIAN)
+            .asIntBuffer();
+
+          Period period = Period.months(intBuf.get(0))
+            .withDays(intBuf.get(1)
+            ).withMillis(intBuf.get(2));
+
+          writer.setPeriod(period);
+        });
+      case FLOAT4:
+        return new ScalarColumnConverter(value -> writer.setDouble((Float) value));
+      case BIT:
+        return new ScalarColumnConverter(value -> writer.setBoolean((Boolean) value));
+      default:
+        return new ScalarColumnConverter(writer::setValue);
+    }
+  }
+
+  public void buildMapMembers(GenericRecord genericRecord, TupleMetadata providedSchema,
+      TupleWriter tupleWriter, List<ColumnConverter> converters) {
+    // fill in tuple schema for cases when it contains recursive named record types
+    TupleMetadata readerSchema = AvroSchemaUtil.convert(genericRecord.getSchema());
+    TupleMetadata tableSchema = StandardConversions.mergeSchemas(providedSchema, readerSchema);
+    tableSchema.toMetadataList().forEach(tupleWriter::addColumn);
+
+    IntStream.range(0, tableSchema.size())
+      .mapToObj(i -> getConverter(providedSchema,
+          readerSchema.metadata(i), tupleWriter.column(i)))
+      .forEach(converters::add);
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/ColumnConvertersUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/ColumnConvertersUtil.java
deleted file mode 100644
index 99308fd..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/ColumnConvertersUtil.java
+++ /dev/null
@@ -1,115 +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.store.avro;
-
-import org.apache.drill.exec.physical.resultSet.RowSetLoader;
-import org.apache.drill.exec.record.metadata.ColumnMetadata;
-import org.apache.drill.exec.record.metadata.TupleMetadata;
-import org.apache.drill.exec.store.avro.ColumnConverter.ArrayColumnConverter;
-import org.apache.drill.exec.store.avro.ColumnConverter.DictColumnConverter;
-import org.apache.drill.exec.store.avro.ColumnConverter.DummyColumnConverter;
-import org.apache.drill.exec.store.avro.ColumnConverter.MapColumnConverter;
-import org.apache.drill.exec.store.avro.ColumnConverter.ScalarColumnConverter;
-import org.apache.drill.exec.vector.accessor.ArrayWriter;
-import org.apache.drill.exec.vector.accessor.DictWriter;
-import org.apache.drill.exec.vector.accessor.ObjectWriter;
-import org.apache.drill.exec.vector.accessor.ScalarWriter;
-import org.apache.drill.exec.vector.accessor.TupleWriter;
-import org.apache.drill.exec.vector.complex.DictVector;
-
-import java.util.List;
-import java.util.stream.Collectors;
-import java.util.stream.IntStream;
-
-public class ColumnConvertersUtil {
-
-  /**
-   * Based on given converted Avro schema and current row writer generates list of
-   * column converters based on column type.
-   *
-   * @param schema converted Avro schema
-   * @param rowWriter current row writer
-   * @return list of column converters
-   */
-  public static List<ColumnConverter> initConverters(TupleMetadata schema, RowSetLoader rowWriter) {
-    return IntStream.range(0, schema.size())
-      .mapToObj(i -> getConverter(schema.metadata(i), rowWriter.column(i)))
-      .collect(Collectors.toList());
-  }
-
-  /**
-   * Based on column type, creates corresponding column converter
-   * which holds conversion logic and appropriate writer to set converted data into.
-   * For columns which are not projected, {@link DummyColumnConverter} is used.
-   *
-   * @param metadata column metadata
-   * @param writer column writer
-   * @return column converter
-   */
-  public static ColumnConverter getConverter(ColumnMetadata metadata, ObjectWriter writer) {
-    if (!writer.isProjected()) {
-      return DummyColumnConverter.INSTANCE;
-    }
-
-    if (metadata.isArray()) {
-      return getArrayConverter(metadata, writer.array());
-    }
-
-    if (metadata.isMap()) {
-      return getMapConverter(metadata.tupleSchema(), writer.tuple());
-    }
-
-    if (metadata.isDict()) {
-      return getDictConverter(metadata.tupleSchema(), writer.dict());
-    }
-
-    return getScalarConverter(writer.scalar());
-  }
-
-  private static ColumnConverter getArrayConverter(ColumnMetadata metadata, ArrayWriter arrayWriter) {
-    ObjectWriter valueWriter = arrayWriter.entry();
-    ColumnConverter valueConverter;
-    if (metadata.isMap()) {
-      valueConverter = getMapConverter(metadata.tupleSchema(), valueWriter.tuple());
-    } else if (metadata.isDict()) {
-      valueConverter = getDictConverter(metadata.tupleSchema(), valueWriter.dict());
-    } else if (metadata.isMultiList()) {
-      valueConverter = getConverter(metadata.childSchema(), valueWriter);
-    } else {
-      valueConverter = getScalarConverter(valueWriter.scalar());
-    }
-    return new ArrayColumnConverter(arrayWriter, valueConverter);
-  }
-
-  private static ColumnConverter getMapConverter(TupleMetadata metadata, TupleWriter tupleWriter) {
-    List<ColumnConverter> converters = IntStream.range(0, metadata.size())
-      .mapToObj(i -> getConverter(metadata.metadata(i), tupleWriter.column(i)))
-      .collect(Collectors.toList());
-    return new MapColumnConverter(tupleWriter, converters);
-  }
-
-  private static ColumnConverter getDictConverter(TupleMetadata metadata, DictWriter dictWriter) {
-    ColumnConverter keyConverter = getScalarConverter(dictWriter.keyWriter());
-    ColumnConverter valueConverter = getConverter(metadata.metadata(DictVector.FIELD_VALUE_NAME), dictWriter.valueWriter());
-    return new DictColumnConverter(dictWriter, keyConverter, valueConverter);
-  }
-
-  private static ColumnConverter getScalarConverter(ScalarWriter scalarWriter) {
-    return ScalarColumnConverter.init(scalarWriter);
-  }
-}
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 22b9b50..1caa55d 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
@@ -68,20 +68,20 @@ import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
- * Base class for various file readers.
+ * Base class for file readers.
  * <p>
- * This version provides a bridge between the legacy {@link RecordReader}-style
+ * Provides a bridge between the legacy {@link RecordReader}-style
  * readers and the newer {@link FileBatchReader} style. Over time, split the
  * class, or provide a cleaner way to handle the differences.
  *
  * @param <T> the format plugin config for this reader
  */
-
 public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements FormatPlugin {
-
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(EasyFormatPlugin.class);
+  private static final Logger logger = LoggerFactory.getLogger(EasyFormatPlugin.class);
 
   /**
    * Defines the static, programmer-defined options for this plugin. These
@@ -89,7 +89,6 @@ public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements
    * defined in the class definition, provides user-defined options that can
    * vary across uses of the plugin.
    */
-
   public static class EasyFormatConfig {
     public BasicFormatMatcher matcher;
     public boolean readable = true;
@@ -111,10 +110,11 @@ public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements
     public int readerOperatorType = -1;
     public int writerOperatorType = -1;
 
-    // Choose whether to use the "traditional" or "enhanced" reader
-    // structure. Can also be selected at runtime by overriding
-    // useEnhancedScan().
-
+    /**
+     *  Choose whether to use the "traditional" or "enhanced" reader
+     *  structure. Can also be selected at runtime by overriding
+     *  {@link #useEnhancedScan()}.
+     */
     public boolean useEnhancedScan;
   }
 
@@ -216,10 +216,9 @@ public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements
    * itself handle the tasks of projecting table columns, creating null
    * columns for missing table columns, and so on?
    *
-   * @return <tt>true</tt> if the plugin supports projection push-down,
-   * <tt>false</tt> if Drill should do the task by adding a project operator
+   * @return {@code true} if the plugin supports projection push-down,
+   * {@code false} if Drill should do the task by adding a project operator
    */
-
   public boolean supportsPushDown() { return easyConfig.supportsProjectPushdown; }
 
   /**
@@ -227,7 +226,7 @@ public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements
    * boundaries. If not, the simple format engine will only split on file
    * boundaries.
    *
-   * @return <code>true</code> if splittable.
+   * @return {@code true} if splittable.
    */
   public boolean isBlockSplittable() { return easyConfig.blockSplittable; }
 
@@ -237,7 +236,7 @@ public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements
    * internal compression scheme, such as Parquet does, then this should return
    * false.
    *
-   * @return <code>true</code> if it is compressible
+   * @return {@code true} if it is compressible
    */
   public boolean isCompressible() { return easyConfig.compressible; }
 
@@ -252,7 +251,6 @@ public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements
    * @return a record reader for this format
    * @throws ExecutionSetupException for many reasons
    */
-
   public RecordReader getRecordReader(FragmentContext context, DrillFileSystem dfs, FileWork fileWork,
       List<SchemaPath> columns, String userName) throws ExecutionSetupException {
     throw new ExecutionSetupException("Must implement getRecordReader() if using the legacy scanner.");
@@ -276,7 +274,6 @@ public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements
    * @return true to use the enhanced scan framework, false for the
    * traditional scan-batch framework
    */
-
   protected boolean useEnhancedScan(OptionManager options) {
     return easyConfig.useEnhancedScan;
   }
@@ -288,7 +285,6 @@ public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements
    * compatibility with "classic" format plugins which have not yet been
    * upgraded to use the new framework.
    */
-
   private CloseableRecordBatch buildScanBatch(FragmentContext context,
       EasySubScan scan) throws ExecutionSetupException {
     final ColumnExplorer columnExplorer =
@@ -342,9 +338,8 @@ public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements
    * Handles most projection tasks automatically. Able to limit
    * vector and batch sizes. Use this for new format plugins.
    */
-
-  private CloseableRecordBatch buildScan(FragmentContext context, EasySubScan scan) throws ExecutionSetupException {
-
+  private CloseableRecordBatch buildScan(FragmentContext context,
+      EasySubScan scan) throws ExecutionSetupException {
     try {
       final FileScanBuilder builder = frameworkBuilder(context.getOptions(), scan);
 
@@ -374,23 +369,24 @@ public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements
    * @param scan the physical scan operator definition passed to
    * the {@link #frameworkBuilder()} method
    */
-
   protected void initScanBuilder(FileScanBuilder builder, EasySubScan scan) {
-    builder.setProjection(scan.getColumns());
-    builder.setFiles(scan.getWorkUnits());
-    builder.setConfig(easyConfig().fsConf);
+    builder.projection(scan.getColumns());
     builder.setUserName(scan.getUserName());
 
     // Pass along the output schema, if any
+    builder.providedSchema(scan.getSchema());
 
-    builder.typeConverterBuilder().providedSchema(scan.getSchema());
+    // Pass along file path information
+    builder.setFileSystemConfig(easyConfig().fsConf);
+    builder.setFiles(scan.getWorkUnits());
     final Path selectionRoot = scan.getSelectionRoot();
     if (selectionRoot != null) {
-      builder.metadataOptions().setSelectionRoot(selectionRoot);
-      builder.metadataOptions().setPartitionDepth(scan.getPartitionDepth());
+      builder.implicitColumnOptions().setSelectionRoot(selectionRoot);
+      builder.implicitColumnOptions().setPartitionDepth(scan.getPartitionDepth());
     }
 
-    builder.setContext(
+    // Additional error context to identify this plugin
+    builder.errorContext(
         new CustomErrorContext() {
           @Override
           public void addContext(UserException.Builder builder) {
@@ -420,7 +416,6 @@ public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements
    * potentially many files
    * @throws ExecutionSetupException for all setup failures
    */
-
   protected FileScanBuilder frameworkBuilder(
       OptionManager options, EasySubScan scan) throws ExecutionSetupException {
     throw new ExecutionSetupException("Must implement frameworkBuilder() if using the enhanced framework.");
@@ -435,8 +430,8 @@ public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements
     throw new UnsupportedOperationException("unimplemented");
   }
 
-  public StatisticsRecordWriter getStatisticsRecordWriter(FragmentContext context, EasyWriter writer) throws IOException
-  {
+  public StatisticsRecordWriter getStatisticsRecordWriter(FragmentContext context,
+      EasyWriter writer) throws IOException {
     return null;
   }
 
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 522fd10..c0523e8 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
@@ -59,7 +59,6 @@ import org.apache.drill.exec.store.easy.text.reader.CompliantTextBatchReader;
 import org.apache.drill.exec.store.easy.text.reader.TextParsingSettings;
 import org.apache.drill.exec.store.easy.text.writer.TextRecordWriter;
 import org.apache.drill.exec.store.schedule.CompleteFileWork;
-import org.apache.drill.exec.vector.accessor.convert.AbstractConvertFromString;
 import org.apache.hadoop.conf.Configuration;
 
 import java.io.IOException;
@@ -105,6 +104,8 @@ public class TextFormatPlugin extends EasyFormatPlugin<TextFormatPlugin.TextForm
   public static final String QUOTE_PROP = TEXT_PREFIX + "quote";
   public static final String QUOTE_ESCAPE_PROP = TEXT_PREFIX + "escape";
   public static final String LINE_DELIM_PROP = TEXT_PREFIX + "lineDelimiter";
+  public static final String TRIM_WHITESPACE_PROP = TEXT_PREFIX + "trim";
+  public static final String PARSE_UNESCAPED_QUOTES_PROP = TEXT_PREFIX + "parseQuotes";
 
   @JsonTypeName(PLUGIN_NAME)
   @JsonInclude(Include.NON_DEFAULT)
@@ -248,37 +249,28 @@ public class TextFormatPlugin extends EasyFormatPlugin<TextFormatPlugin.TextForm
       OptionManager options, EasySubScan scan) throws ExecutionSetupException {
     ColumnsScanBuilder builder = new ColumnsScanBuilder();
     initScanBuilder(builder, scan);
+
     TextParsingSettings settings =
-        new TextParsingSettings(getConfig(), scan, options);
+        new TextParsingSettings(getConfig(), scan.getSchema());
     builder.setReaderFactory(new ColumnsReaderFactory(settings));
 
     // If this format has no headers, or wants to skip them,
     // then we must use the columns column to hold the data.
-
-    builder.requireColumnsArray(settings.isUseRepeatedVarChar());
+    builder.requireColumnsArray(
+        ! settings.isHeaderExtractionEnabled() && builder.providedSchema() == null);
 
     // Text files handle nulls in an unusual way. Missing columns
     // are set to required Varchar and filled with blanks. Yes, this
     // means that the SQL statement or code cannot differentiate missing
     // columns from empty columns, but that is how CSV and other text
     // files have been defined within Drill.
-
-    builder.setNullType(Types.required(MinorType.VARCHAR));
-
-    // CSV maps blank columns to nulls (for nullable non-string columns),
-    // or to the default value (for non-nullable non-string columns.)
-
-    builder.typeConverterBuilder().setConversionProperty(
-        AbstractConvertFromString.BLANK_ACTION_PROP,
-        AbstractConvertFromString.BLANK_AS_NULL);
+    builder.nullType(Types.required(MinorType.VARCHAR));
 
     // The text readers use required Varchar columns to represent null columns.
-
     builder.allowRequiredNullColumns(true);
 
     // Provide custom error context
-
-    builder.setContext(
+    builder.errorContext(
         new ChildErrorContext(builder.errorContext()) {
           @Override
           public void addContext(UserException.Builder builder) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/reader/BaseFieldOutput.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/reader/BaseFieldOutput.java
index d4edaf2..bd4d852 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/reader/BaseFieldOutput.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/reader/BaseFieldOutput.java
@@ -18,7 +18,7 @@
 package org.apache.drill.exec.store.easy.text.reader;
 
 import org.apache.drill.exec.physical.resultSet.RowSetLoader;
-import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.ValueWriter;
 
 public abstract class BaseFieldOutput implements TextOutput {
 
@@ -63,7 +63,6 @@ public abstract class BaseFieldOutput implements TextOutput {
    * @param projectionMask a boolean array indicating which fields are
    * to be projected to the output. Optional
    */
-
   public BaseFieldOutput(RowSetLoader writer, int maxField, boolean[] projectionMask) {
     this.writer = writer;
     this.projectionMask = projectionMask;
@@ -97,7 +96,6 @@ public abstract class BaseFieldOutput implements TextOutput {
   /**
    * Start a new record record. Resets all pointers
    */
-
   @Override
   public void startRecord() {
     currentFieldIndex = -1;
@@ -136,7 +134,6 @@ public abstract class BaseFieldOutput implements TextOutput {
     fieldBytes[currentDataPointer++] = data;
   }
 
-
   /**
    * Write a buffer of data to the underlying vector using the
    * column writer. The buffer holds a complete or partial chunk
@@ -147,12 +144,11 @@ public abstract class BaseFieldOutput implements TextOutput {
    * (This is generally OK because the previous setBytes should have
    * failed because a large int or date is not supported.)
    */
-
   protected void writeToVector() {
     if (!fieldProjected) {
       return;
     }
-    ScalarWriter colWriter = columnWriter();
+    ValueWriter colWriter = columnWriter();
     if (fieldWriteCount == 0) {
       colWriter.setBytes(fieldBytes, currentDataPointer);
     } else {
@@ -162,7 +158,7 @@ public abstract class BaseFieldOutput implements TextOutput {
     currentDataPointer = 0;
   }
 
-  protected abstract ScalarWriter columnWriter();
+  protected abstract ValueWriter columnWriter();
 
   @Override
   public boolean endField() {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/reader/CompliantTextBatchReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/reader/CompliantTextBatchReader.java
index f1512a8..8e3e69e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/reader/CompliantTextBatchReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/reader/CompliantTextBatchReader.java
@@ -19,6 +19,8 @@ package org.apache.drill.exec.store.easy.text.reader;
 
 import java.io.IOException;
 import java.io.InputStream;
+import java.util.HashMap;
+import java.util.Map;
 
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.types.TypeProtos.DataMode;
@@ -26,12 +28,16 @@ import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.ops.OperatorContext;
 import org.apache.drill.exec.physical.impl.scan.columns.ColumnsScanFramework;
 import org.apache.drill.exec.physical.impl.scan.columns.ColumnsSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.convert.StandardConversions;
 import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
 import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.apache.drill.exec.record.metadata.MetadataUtils;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.record.metadata.TupleSchema;
 import org.apache.drill.exec.store.dfs.DrillFileSystem;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.ValueWriter;
 import org.apache.hadoop.mapred.FileSplit;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -41,7 +47,7 @@ import com.univocity.parsers.common.TextParsingException;
 import io.netty.buffer.DrillBuf;
 
 /**
- * New text reader, complies with the RFC 4180 standard for text/csv files
+ * Text reader, Complies with the RFC 4180 standard for text/csv files.
  */
 public class CompliantTextBatchReader implements ManagedReader<ColumnsSchemaNegotiator> {
   private static final Logger logger = LoggerFactory.getLogger(CompliantTextBatchReader.class);
@@ -58,7 +64,7 @@ public class CompliantTextBatchReader implements ManagedReader<ColumnsSchemaNego
   private TextReader reader;
   // input buffer
   private DrillBuf readBuffer;
-  // working buffer to handle whitespaces
+  // working buffer to handle whitespace
   private DrillBuf whitespaceBuffer;
   private DrillFileSystem dfs;
 
@@ -69,7 +75,6 @@ public class CompliantTextBatchReader implements ManagedReader<ColumnsSchemaNego
 
     // Validate. Otherwise, these problems show up later as a data
     // read error which is very confusing.
-
     if (settings.getNewLineDelimiter().length == 0) {
       throw UserException
         .validationError()
@@ -85,7 +90,6 @@ public class CompliantTextBatchReader implements ManagedReader<ColumnsSchemaNego
    * @param errorContext  operator context from which buffer's will be allocated and managed
    * @param outputMutator  Used to create the schema in the output record batch
    */
-
   @Override
   public boolean open(ColumnsSchemaNegotiator schemaNegotiator) {
     final OperatorContext context = schemaNegotiator.context();
@@ -101,16 +105,15 @@ public class CompliantTextBatchReader implements ManagedReader<ColumnsSchemaNego
 
     readBuffer = context.getAllocator().buffer(READ_BUFFER);
     whitespaceBuffer = context.getAllocator().buffer(WHITE_SPACE_BUFFER);
-    schemaNegotiator.setBatchSize(MAX_RECORDS_PER_BATCH);
+    schemaNegotiator.batchSize(MAX_RECORDS_PER_BATCH);
 
     // setup Output, Input, and Reader
     try {
       TextOutput output;
-
-      if (settings.isUseRepeatedVarChar()) {
-        output = openWithoutHeaders(schemaNegotiator);
+      if (settings.isHeaderExtractionEnabled()) {
+        output = openWithHeaders(schemaNegotiator);
       } else {
-        output = openWithHeaders(schemaNegotiator, settings.providedHeaders());
+        output = openWithoutHeaders(schemaNegotiator);
       }
       if (output == null) {
         return false;
@@ -131,29 +134,101 @@ public class CompliantTextBatchReader implements ManagedReader<ColumnsSchemaNego
    * with a text file that contains no headers; ignored for
    * text file with headers
    */
-
-  private TextOutput openWithHeaders(ColumnsSchemaNegotiator schemaNegotiator,
-      String[] providedHeaders) throws IOException {
-    final String [] fieldNames = providedHeaders == null ? extractHeader() : providedHeaders;
+  private TextOutput openWithHeaders(ColumnsSchemaNegotiator schemaNegotiator) throws IOException {
+    final String [] fieldNames = extractHeader();
     if (fieldNames == null) {
       return null;
     }
+    if (schemaNegotiator.hasProvidedSchema()) {
+      return buildWithSchema(schemaNegotiator, fieldNames);
+    } else {
+      return buildFromColumnHeaders(schemaNegotiator, fieldNames);
+    }
+  }
+
+  /**
+   * File has headers and a provided schema is provided. Convert from VARCHAR
+   * input type to the provided output type, but only if the column is projected.
+   */
+  private FieldVarCharOutput buildWithSchema(ColumnsSchemaNegotiator schemaNegotiator,
+      String[] fieldNames) {
+    TupleMetadata readerSchema = mergeSchemas(schemaNegotiator.providedSchema(), fieldNames);
+    schemaNegotiator.tableSchema(readerSchema, true);
+    writer = schemaNegotiator.build().writer();
+    StandardConversions conversions = conversions(schemaNegotiator.providedSchema());
+    ValueWriter[] colWriters = new ValueWriter[fieldNames.length];
+    for (int i = 0; i < fieldNames.length; i++) {
+      ScalarWriter colWriter = writer.scalar(fieldNames[i]);
+      if (writer.isProjected()) {
+        colWriters[i] = conversions.converter(colWriter, MinorType.VARCHAR);
+      } else {
+        colWriters[i] = colWriter;
+      }
+    }
+    return new FieldVarCharOutput(writer, colWriters);
+  }
+
+  private TupleMetadata mergeSchemas(TupleMetadata providedSchema,
+      String[] fieldNames) {
+    final TupleMetadata readerSchema = new TupleSchema();
+    for (String fieldName : fieldNames) {
+      final ColumnMetadata providedCol = providedSchema.metadata(fieldName);
+      readerSchema.addColumn(providedCol == null ? textColumn(fieldName) : providedCol);
+    }
+    return readerSchema;
+  }
+
+  private ColumnMetadata textColumn(String colName) {
+    return MetadataUtils.newScalar(colName, MinorType.VARCHAR, DataMode.REQUIRED);
+  }
+
+  /**
+   * File has column headers. No provided schema. Build schema from the
+   * column headers.
+   */
+  private FieldVarCharOutput buildFromColumnHeaders(ColumnsSchemaNegotiator schemaNegotiator,
+      String[] fieldNames) {
     final TupleMetadata schema = new TupleSchema();
     for (final String colName : fieldNames) {
-      schema.addColumn(MetadataUtils.newScalar(colName, MinorType.VARCHAR, DataMode.REQUIRED));
+      schema.addColumn(textColumn(colName));
     }
-    schemaNegotiator.setTableSchema(schema, true);
+    schemaNegotiator.tableSchema(schema, true);
     writer = schemaNegotiator.build().writer();
-    return new FieldVarCharOutput(writer);
+    ValueWriter[] colWriters = new ValueWriter[fieldNames.length];
+    for (int i = 0; i < fieldNames.length; i++) {
+      colWriters[i] = writer.column(i).scalar();
+    }
+    return new FieldVarCharOutput(writer, colWriters);
   }
 
   /**
    * When no headers, create a single array column "columns".
    */
-
   private TextOutput openWithoutHeaders(
       ColumnsSchemaNegotiator schemaNegotiator) {
-    schemaNegotiator.setTableSchema(ColumnsScanFramework.columnsSchema(), true);
+    if (schemaNegotiator.hasProvidedSchema()) {
+      return buildWithSchema(schemaNegotiator);
+    } else {
+      return buildColumnsArray(schemaNegotiator);
+    }
+  }
+
+  private FieldVarCharOutput buildWithSchema(ColumnsSchemaNegotiator schemaNegotiator) {
+    TupleMetadata providedSchema = schemaNegotiator.providedSchema();
+    schemaNegotiator.tableSchema(providedSchema, true);
+    writer = schemaNegotiator.build().writer();
+    StandardConversions conversions = conversions(providedSchema);
+    ValueWriter[] colWriters = new ValueWriter[providedSchema.size()];
+    for (int i = 0; i < colWriters.length; i++) {
+      colWriters[i] = conversions.converter(
+          writer.scalar(providedSchema.metadata(i).name()), MinorType.VARCHAR);
+    }
+    return new ConstrainedFieldOutput(writer, colWriters);
+  }
+
+  private TextOutput buildColumnsArray(
+      ColumnsSchemaNegotiator schemaNegotiator) {
+    schemaNegotiator.tableSchema(ColumnsScanFramework.columnsSchema(), true);
     writer = schemaNegotiator.build().writer();
     return new RepeatedVarCharOutput(writer, schemaNegotiator.projectedIndexes());
   }
@@ -169,13 +244,26 @@ public class CompliantTextBatchReader implements ManagedReader<ColumnsSchemaNego
     reader.start();
   }
 
+  private StandardConversions conversions(TupleMetadata providedSchema) {
+
+    // CSV maps blank columns to nulls (for nullable non-string columns),
+    // or to the default value (for non-nullable non-string columns.)
+    Map<String, String> props = providedSchema.properties();
+    if (props == null) {
+      return new StandardConversions(ColumnMetadata.BLANK_AS_NULL);
+    } else {
+      props = new HashMap<>(props);
+      props.put(ColumnMetadata.BLANK_AS_PROP, ColumnMetadata.BLANK_AS_NULL);
+      return new StandardConversions(props);
+    }
+  }
+
   /**
    * Extracts header from text file.
    * Currently it is assumed to be first line if headerExtractionEnabled is set to true
    * TODO: enhance to support more common header patterns
    * @return field name strings
    */
-
   private String[] extractHeader() throws IOException {
     assert settings.isHeaderExtractionEnabled();
 
@@ -212,7 +300,6 @@ public class CompliantTextBatchReader implements ManagedReader<ColumnsSchemaNego
    * Generates the next record batch
    * @return  number of records in the batch
    */
-
   @Override
   public boolean next() {
     reader.resetForNextBatch();
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertIntToString.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/reader/ConstrainedFieldOutput.java
similarity index 53%
rename from exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertIntToString.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/reader/ConstrainedFieldOutput.java
index b869ea5..5078a57 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/convert/ConvertIntToString.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/reader/ConstrainedFieldOutput.java
@@ -15,29 +15,30 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.vector.accessor.convert;
+package org.apache.drill.exec.store.easy.text.reader;
 
-import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.vector.accessor.ValueWriter;
 
-// TODO: Provide some kind of format
-
-public class ConvertIntToString extends AbstractWriteConverter {
+/**
+ * For CSV files without headers, but with a provided schema,
+ * handles the case where extra fields appear in the file beyond
+ * the columns enumerated in the schema. These fields are ignored.
+ */
+public class ConstrainedFieldOutput extends FieldVarCharOutput {
 
-  public ConvertIntToString(ScalarWriter baseWriter) {
-    super(baseWriter);
+  ConstrainedFieldOutput(RowSetLoader writer, ValueWriter[] colWriters) {
+    super(writer, colWriters);
   }
 
   @Override
-  public void setInt(int value) {
-    baseWriter.setString(Integer.toString(value));
-  }
+  protected void writeToVector() {
 
-  @Override
-  public void setValue(Object value) {
-    if (value == null) {
-      setNull();
+    // Reject columns past the known schema.
+    if (currentFieldIndex < colWriters.length) {
+      super.writeToVector();
     } else {
-      setInt((int) value);
+      currentDataPointer = 0;
     }
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/reader/FieldVarCharOutput.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/reader/FieldVarCharOutput.java
index 7fe6bd5..f8e44f7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/reader/FieldVarCharOutput.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/reader/FieldVarCharOutput.java
@@ -20,25 +20,29 @@ package org.apache.drill.exec.store.easy.text.reader;
 import org.apache.drill.exec.physical.resultSet.RowSetLoader;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.store.easy.text.TextFormatPlugin;
-import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.ValueWriter;
 
 /**
- * Class is responsible for generating record batches for text file inputs. We generate
- * a record batch with a set of varchar vectors. A varchar vector contains all the field
- * values for a given column. Each record is a single value within each vector of the set.
+ * Class is responsible for generating record batches for text file inputs. We
+ * generate a record batch with a set of varchar vectors. A varchar vector
+ * contains all the field values for a given column. Each record is a single
+ * value within each vector of the set.
  */
 class FieldVarCharOutput extends BaseFieldOutput {
 
+  protected final ValueWriter[] colWriters;
+
   /**
    * We initialize and add the varchar vector for each incoming field in this
    * constructor.
    *
    * @param writer row set writer
    */
-  FieldVarCharOutput(RowSetLoader writer) {
+  FieldVarCharOutput(RowSetLoader writer, ValueWriter[] colWriters) {
     super(writer,
         TextFormatPlugin.MAXIMUM_NUMBER_COLUMNS,
         makeMask(writer));
+    this.colWriters = colWriters;
   }
 
   private static boolean[] makeMask(RowSetLoader writer) {
@@ -57,7 +61,7 @@ class FieldVarCharOutput extends BaseFieldOutput {
   }
 
   @Override
-  protected ScalarWriter columnWriter() {
-    return writer.scalar(currentFieldIndex);
+  protected ValueWriter columnWriter() {
+    return colWriters[currentFieldIndex];
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/reader/TextInput.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/reader/TextInput.java
index 0ca2cfe..51ed78f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/reader/TextInput.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/reader/TextInput.java
@@ -47,7 +47,7 @@ final class TextInput {
   private long charCount;
 
   /**
-   * The starting position in the file.
+   * Starting position in the file.
    */
   private final long startPos;
   private final long endPos;
@@ -72,16 +72,16 @@ final class TextInput {
   private int remByte = -1;
 
   /**
-   * The current position in the buffer.
+   * Current position in the buffer.
    */
   private int bufferPtr;
 
   /**
-   * The quantity of valid data in the buffer.
+   * Length of valid data in the buffer.
    */
   private int length = -1;
 
-  private boolean endFound = false;
+  private boolean endFound;
 
   /**
    * Creates a new instance with the mandatory characters for handling newlines
@@ -91,6 +91,8 @@ final class TextInput {
    * {@link TextParsingSettings#getNormalizedNewLine()}) that is used to replace any
    * lineSeparator sequence found in the input.
    */
+  // TODO: Remove the DrillBuf; we're getting no benefit from the round trip
+  // out to direct memory and back.
   TextInput(TextParsingSettings settings, InputStream input, DrillBuf readBuffer, long startPos, long endPos) {
     this.lineSeparator = settings.getNewLineDelimiter();
     byte normalizedLineSeparator = settings.getNormalizedNewLine();
@@ -98,7 +100,8 @@ final class TextInput {
     boolean isCompressed = input instanceof CompressionInputStream;
     Preconditions.checkArgument(!isCompressed || startPos == 0, "Cannot use split on compressed stream.");
 
-    // splits aren't allowed with compressed data.  The split length will be the compressed size which means we'll normally end prematurely.
+    // splits aren't allowed with compressed data.  The split length will be the
+    // compressed size which means we'll normally end prematurely.
     if (isCompressed && endPos > 0) {
       endPos = Long.MAX_VALUE;
     }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/reader/TextParsingSettings.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/reader/TextParsingSettings.java
index acc1cda..8fbcd9b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/reader/TextParsingSettings.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/reader/TextParsingSettings.java
@@ -18,16 +18,12 @@
 package org.apache.drill.exec.store.easy.text.reader;
 
 import org.apache.drill.exec.record.metadata.TupleMetadata;
-import org.apache.drill.exec.server.options.OptionManager;
-import org.apache.drill.exec.store.dfs.easy.EasySubScan;
 import org.apache.drill.exec.store.easy.text.TextFormatPlugin;
 import org.apache.drill.exec.store.easy.text.TextFormatPlugin.TextFormatConfig;
 import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
 
 public class TextParsingSettings {
 
-  private final String emptyValue = null;
-  private final boolean parseUnescapedQuotes = true;
   private final byte quote;
   private final byte quoteEscape;
   private final byte delimiter;
@@ -36,14 +32,14 @@ public class TextParsingSettings {
   private final long maxCharsPerColumn = TextFormatPlugin.MAX_CHARS_PER_COLUMN;
   private final byte normalizedNewLine = b('\n');
   private final byte[] newLineDelimiter;
-  private final boolean ignoreLeadingWhitespaces = false;
-  private final boolean ignoreTrailingWhitespaces = false;
   private final String lineSeparatorString = "\n";
   private boolean skipFirstLine;
-
   private final boolean headerExtractionEnabled;
-  private final boolean useRepeatedVarChar;
-  private final String providedHeaders[];
+
+  // Available only via table properties
+  private final boolean parseUnescapedQuotes;
+  private final boolean ignoreLeadingWhitespace;
+  private final boolean ignoreTrailingWhitespace;
 
   /**
    * Configure the properties for this one scan based on:
@@ -65,11 +61,12 @@ public class TextParsingSettings {
    * file with headers, the user can just customize the table properties.
    */
   public TextParsingSettings(TextFormatConfig config,
-      EasySubScan scan, OptionManager options) {
-    TupleMetadata providedSchema = scan.getSchema();
+      TupleMetadata providedSchema) {
     boolean extractHeaders = config.isHeaderExtractionEnabled();
     boolean skipFirst = config.isSkipFirstLine();
-    String providedHeaders[] = null;
+    boolean ignoreLeadingWhitespace = false;
+    boolean ignoreTrailingWhitespace = false;
+    boolean parseUnescapedQuotes = true;
     byte delimChar = bSafe(config.getFieldDelimiter(), "fieldDelimiter");
     byte commentChar = bSafe(config.getComment(), "comment");
     byte quoteChar = bSafe(config.getQuote(), "quote");
@@ -80,28 +77,31 @@ public class TextParsingSettings {
           TextFormatPlugin.HAS_HEADERS_PROP, extractHeaders);
       skipFirst = ! extractHeaders & providedSchema.booleanProperty(
           TextFormatPlugin.SKIP_FIRST_LINE_PROP, skipFirstLine);
-      if (!extractHeaders && ! providedSchema.isEmpty()) {
-        providedHeaders = new String[providedSchema.size()];
-        for (int i = 0; i < providedHeaders.length; i++) {
-          providedHeaders[i] = providedSchema.metadata(i).name();
-        }
-      }
+      skipFirst = ! extractHeaders & providedSchema.booleanProperty(
+          TextFormatPlugin.SKIP_FIRST_LINE_PROP, skipFirstLine);
+      ignoreLeadingWhitespace = providedSchema.booleanProperty(
+          TextFormatPlugin.TRIM_WHITESPACE_PROP, ignoreLeadingWhitespace);
+      ignoreTrailingWhitespace = providedSchema.booleanProperty(
+          TextFormatPlugin.TRIM_WHITESPACE_PROP, ignoreTrailingWhitespace);
+      parseUnescapedQuotes = providedSchema.booleanProperty(
+          TextFormatPlugin.PARSE_UNESCAPED_QUOTES_PROP, parseUnescapedQuotes);
       delimChar = overrideChar(providedSchema, TextFormatPlugin.DELIMITER_PROP, delimChar);
       quoteChar = overrideChar(providedSchema, TextFormatPlugin.QUOTE_PROP, quoteChar);
       quoteEscapeChar = overrideChar(providedSchema, TextFormatPlugin.QUOTE_ESCAPE_PROP, quoteEscapeChar);
       newlineDelim = newlineDelimBytes(providedSchema, newlineDelim);
       commentChar = commentChar(providedSchema, commentChar);
     }
-    skipFirstLine = !extractHeaders && skipFirst;
-    headerExtractionEnabled = extractHeaders;
-    this.providedHeaders = providedHeaders;
-    useRepeatedVarChar = !extractHeaders && providedHeaders == null;
+    this.skipFirstLine = !extractHeaders && skipFirst;
+    this.headerExtractionEnabled = extractHeaders;
 
-    quote = quoteChar;
-    quoteEscape = quoteEscapeChar;
-    newLineDelimiter = newlineDelim;
-    delimiter = delimChar;
-    comment = commentChar;
+    this.quote = quoteChar;
+    this.quoteEscape = quoteEscapeChar;
+    this.newLineDelimiter = newlineDelim;
+    this.delimiter = delimChar;
+    this.comment = commentChar;
+    this.ignoreLeadingWhitespace = ignoreLeadingWhitespace;
+    this.ignoreTrailingWhitespace = ignoreTrailingWhitespace;
+    this.parseUnescapedQuotes = parseUnescapedQuotes;
   }
 
   /**
@@ -109,7 +109,6 @@ public class TextParsingSettings {
    * or is a blank string, then uses the delimiter from the plugin config.
    * Else, if non-blank, uses the first character of the property value.
    */
-
   private static byte overrideChar(TupleMetadata providedSchema, String propName, byte configValue) {
     String value = providedSchema.property(propName);
     if (value == null || value.isEmpty()) {
@@ -126,7 +125,6 @@ public class TextParsingSettings {
    * match anything, and effectively disables the comment feature.
    * Else, if non-blank, uses the first character of the property value.
    */
-
   private static byte commentChar(TupleMetadata providedSchema, byte configValue) {
     String value = providedSchema.property(TextFormatPlugin.COMMENT_CHAR_PROP);
     if (value == null) {
@@ -164,10 +162,6 @@ public class TextParsingSettings {
     this.skipFirstLine = skipFirstLine;
   }
 
-  public boolean isUseRepeatedVarChar() {
-    return useRepeatedVarChar;
-  }
-
   private static byte bSafe(char c, String name) {
     if (c > Byte.MAX_VALUE) {
       throw new IllegalArgumentException(String.format("Failure validating configuration option %s.  Expected a "
@@ -215,26 +209,13 @@ public class TextParsingSettings {
   }
 
   /**
-   * Returns the String representation of an empty value (defaults to null)
-   *
-   * <p>
-   * When reading, if the parser does not read any character from the input, and
-   * the input is within quotes, the empty is used instead of an empty string
-   *
-   * @return the String representation of an empty value
-   */
-  public String getEmptyValue() {
-    return emptyValue;
-  }
-
-  /**
    * Indicates whether the CSV parser should accept unescaped quotes inside
    * quoted values and parse them normally. Defaults to {@code true}.
    *
    * @return a flag indicating whether or not the CSV parser should accept
    *         unescaped quotes inside quoted values.
    */
-  public boolean isParseUnescapedQuotes() {
+  public boolean parseUnescapedQuotes() {
     return parseUnescapedQuotes;
   }
 
@@ -258,13 +239,11 @@ public class TextParsingSettings {
     return normalizedNewLine;
   }
 
-  public boolean isIgnoreLeadingWhitespaces() {
-    return ignoreLeadingWhitespaces;
+  public boolean ignoreLeadingWhitespace() {
+    return ignoreLeadingWhitespace;
   }
 
-  public boolean isIgnoreTrailingWhitespaces() {
-    return ignoreTrailingWhitespaces;
+  public boolean ignoreTrailingWhitespace() {
+    return ignoreTrailingWhitespace;
   }
-
-  public String[] providedHeaders() { return providedHeaders; }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/reader/TextReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/reader/TextReader.java
index 96d62f3..e3a5edd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/reader/TextReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/reader/TextReader.java
@@ -82,9 +82,9 @@ public final class TextReader {
     this.workBuf = workBuf;
     this.settings = settings;
 
-    this.ignoreTrailingWhitespace = settings.isIgnoreTrailingWhitespaces();
-    this.ignoreLeadingWhitespace = settings.isIgnoreLeadingWhitespaces();
-    this.parseUnescapedQuotes = settings.isParseUnescapedQuotes();
+    this.ignoreTrailingWhitespace = settings.ignoreTrailingWhitespace();
+    this.ignoreLeadingWhitespace = settings.ignoreLeadingWhitespace();
+    this.parseUnescapedQuotes = settings.parseUnescapedQuotes();
     this.delimiter = settings.getDelimiter();
     this.quote = settings.getQuote();
     this.quoteEscape = settings.getQuoteEscape();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/log/LogBatchReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/log/LogBatchReader.java
index 0ed71db..f8c512e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/log/LogBatchReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/log/LogBatchReader.java
@@ -26,20 +26,22 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.physical.impl.scan.convert.StandardConversions;
 import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
 import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
 import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
 import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.vector.accessor.ScalarWriter;
 import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.drill.exec.vector.accessor.ValueWriter;
 import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
 import org.apache.hadoop.mapred.FileSplit;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 public class LogBatchReader implements ManagedReader<FileSchemaNegotiator> {
-
   private static final Logger logger = LoggerFactory.getLogger(LogBatchReader.class);
   public static final String RAW_LINE_COL_NAME = "_raw";
   public static final String UNMATCHED_LINE_COL_NAME = "_unmatched_rows";
@@ -47,17 +49,22 @@ public class LogBatchReader implements ManagedReader<FileSchemaNegotiator> {
   public static class LogReaderConfig {
     protected final LogFormatPlugin plugin;
     protected final Pattern pattern;
-    protected final TupleMetadata schema;
+    protected final TupleMetadata providedSchema;
+    protected final TupleMetadata tableSchema;
+    protected final TupleMetadata readerSchema;
     protected final boolean asArray;
     protected final int groupCount;
     protected final int maxErrors;
 
     public LogReaderConfig(LogFormatPlugin plugin, Pattern pattern,
-        TupleMetadata schema, boolean asArray,
-        int groupCount, int maxErrors) {
+        TupleMetadata providedSchema, TupleMetadata tableSchema,
+        TupleMetadata readerSchema,
+        boolean asArray, int groupCount, int maxErrors) {
       this.plugin = plugin;
       this.pattern = pattern;
-      this.schema = schema;
+      this.providedSchema = providedSchema;
+      this.tableSchema = tableSchema;
+      this.readerSchema = readerSchema;
       this.asArray = asArray;
       this.groupCount = groupCount;
       this.maxErrors = maxErrors;
@@ -67,7 +74,6 @@ public class LogBatchReader implements ManagedReader<FileSchemaNegotiator> {
   /**
    * Write group values to value vectors.
    */
-
   private interface VectorWriter {
     void loadVectors(Matcher m);
   }
@@ -75,13 +81,17 @@ public class LogBatchReader implements ManagedReader<FileSchemaNegotiator> {
   /**
    * Write group values to individual scalar columns.
    */
-
   private static class ScalarGroupWriter implements VectorWriter {
 
-    private final TupleWriter rowWriter;
+    private final ValueWriter[] writers;
 
-    public ScalarGroupWriter(TupleWriter rowWriter) {
-      this.rowWriter = rowWriter;
+    public ScalarGroupWriter(TupleWriter rowWriter,
+        TupleMetadata readerSchema, StandardConversions conversions) {
+      writers = new ValueWriter[readerSchema.size()];
+      for (int i = 0; i < writers.length; i++) {
+        ColumnMetadata colSchema = readerSchema.metadata(i);
+        writers[i] = conversions.converter(rowWriter.scalar(i), colSchema);
+      }
     }
 
     @Override
@@ -89,7 +99,7 @@ public class LogBatchReader implements ManagedReader<FileSchemaNegotiator> {
       for (int i = 0; i < m.groupCount(); i++) {
         String value = m.group(i + 1);
         if (value != null) {
-          rowWriter.scalar(i).setString(value);
+          writers[i].setString(value);
         }
       }
     }
@@ -98,7 +108,6 @@ public class LogBatchReader implements ManagedReader<FileSchemaNegotiator> {
   /**
    * Write group values to the columns[] array.
    */
-
   private static class ColumnsArrayWriter implements VectorWriter {
 
     private final ScalarWriter elementWriter;
@@ -134,7 +143,7 @@ public class LogBatchReader implements ManagedReader<FileSchemaNegotiator> {
   @Override
   public boolean open(FileSchemaNegotiator negotiator) {
     split = negotiator.split();
-    negotiator.setTableSchema(config.schema, true);
+    negotiator.tableSchema(config.tableSchema, true);
     loader = negotiator.build();
     bindColumns(loader.writer());
     openFile(negotiator);
@@ -149,7 +158,6 @@ public class LogBatchReader implements ManagedReader<FileSchemaNegotiator> {
     // If no match-case columns are projected, and the unmatched
     // columns is unprojected, then we want to count (matched)
     // rows.
-
     saveMatchedRows |= !unmatchedColWriter.isProjected();
 
     // This reader is unusual: it can save only unmatched rows,
@@ -157,20 +165,23 @@ public class LogBatchReader implements ManagedReader<FileSchemaNegotiator> {
     // save matched rows to by checking if any of the "normal"
     // reader columns are projected (ignoring the two special
     // columns.) If so, create a vector writer to save values.
-
     if (config.asArray) {
       saveMatchedRows |= writer.column(0).isProjected();
       if (saveMatchedRows) {
-        // Save using the defined columns
+         // Save columns as an array
         vectorWriter = new ColumnsArrayWriter(writer);
       }
     } else {
-      for (int i = 0; i <  config.schema.size(); i++) {
+      for (int i = 0; i <  config.readerSchema.size(); i++) {
         saveMatchedRows |= writer.column(i).isProjected();
       }
       if (saveMatchedRows) {
-        // Save columns as an array
-        vectorWriter = new ScalarGroupWriter(writer);
+        // Save using the defined columns
+        TupleMetadata providedSchema = config.providedSchema;
+        StandardConversions conversions = new StandardConversions(
+            providedSchema == null || !providedSchema.hasProperties() ?
+                null : providedSchema.properties());
+        vectorWriter = new ScalarGroupWriter(writer, config.readerSchema, conversions);
       }
     }
   }
@@ -184,7 +195,7 @@ public class LogBatchReader implements ManagedReader<FileSchemaNegotiator> {
           .dataReadError(e)
           .message("Failed to open input file")
           .addContext("File path:", split.getPath())
-          .addContext(loader.context())
+          .addContext(loader.errorContext())
           .build(logger);
     }
     reader = new BufferedReader(new InputStreamReader(in, Charsets.UTF_8));
@@ -210,7 +221,7 @@ public class LogBatchReader implements ManagedReader<FileSchemaNegotiator> {
           .dataReadError(e)
           .message("Error reading file")
           .addContext("File", split.getPath())
-          .addContext(loader.context())
+          .addContext(loader.errorContext())
           .build(logger);
     }
 
@@ -222,7 +233,6 @@ public class LogBatchReader implements ManagedReader<FileSchemaNegotiator> {
     if (lineMatcher.matches()) {
 
       // Load matched row into vectors.
-
       if (saveMatchedRows) {
         rowWriter.start();
         rawColWriter.setString(line);
@@ -240,13 +250,12 @@ public class LogBatchReader implements ManagedReader<FileSchemaNegotiator> {
           .message("Too many errors. Max error threshold exceeded.")
           .addContext("Line", line)
           .addContext("Line number", lineNumber)
-          .addContext(loader.context())
+          .addContext(loader.errorContext())
           .build(logger);
     }
 
     // For unmatched columns, create an output row only if the
     // user asked for the unmatched values.
-
     if (unmatchedColWriter.isProjected()) {
       rowWriter.start();
       unmatchedColWriter.setString(line);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/log/LogFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/log/LogFormatPlugin.java
index ce017cd..8a3a6af 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/log/LogFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/log/LogFormatPlugin.java
@@ -30,13 +30,13 @@ import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.physical.impl.scan.columns.ColumnsScanFramework;
-import org.apache.drill.exec.physical.impl.scan.columns.ColumnsScanFramework.ColumnsScanBuilder;
 import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileReaderFactory;
 import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileScanBuilder;
 import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
 import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
 import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
 import org.apache.drill.exec.record.metadata.Propertied;
 import org.apache.drill.exec.record.metadata.SchemaBuilder;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
@@ -139,22 +139,19 @@ public class LogFormatPlugin extends EasyFormatPlugin<LogFormatConfig> {
    * suffix.</li>
    * </ul>
    */
-
   @Override
   protected FileScanBuilder frameworkBuilder(
       OptionManager options, EasySubScan scan) throws ExecutionSetupException {
 
     // Pattern and schema identical across readers; define
     // up front.
-
-    TupleMetadata providedSchema = scan.getSchema();
-    Pattern pattern = setupPattern(providedSchema);
+    final TupleMetadata providedSchema = scan.getSchema();
+    final Pattern pattern = setupPattern(providedSchema);
 
     // Use a dummy matcher to get the group count. Group count not
     // available from the pattern itself, oddly.
-
-    Matcher m = pattern.matcher("dummy");
-    int groupCount = m.groupCount();
+    final Matcher m = pattern.matcher("dummy");
+    final int groupCount = m.groupCount();
     if (groupCount == 0) {
       throw UserException
         .validationError()
@@ -163,63 +160,58 @@ public class LogFormatPlugin extends EasyFormatPlugin<LogFormatConfig> {
         .build(logger);
     }
 
-    boolean hasColumns = (providedSchema != null && providedSchema.size() > 0);
-    boolean hasSchema = hasColumns || formatConfig.hasSchema();
-    SchemaBuilder schemaBuilder = new SchemaBuilder();
-    FileScanBuilder builder;
+    final boolean hasColumns = (providedSchema != null && providedSchema.size() > 0);
+    final boolean hasSchema = hasColumns || formatConfig.hasSchema();
+    final TupleMetadata readerSchema;
+    final TupleMetadata tableSchema;
+
+    // Use the file framework to enable support for implicit and partition
+    // columns.
+    final FileScanBuilder builder = new FileScanBuilder();
+    initScanBuilder(builder, scan);
     if (hasSchema) {
-      TupleMetadata outputSchema;
       if (!hasColumns) {
 
         // No provided schema, build from plugin config
         // (or table function.)
-
-        outputSchema = defineOutputSchemaFromConfig(groupCount);
+        tableSchema = defineOutputSchemaFromConfig(groupCount);
+        readerSchema = defineReaderSchema(tableSchema);
       } else if (groupCount <= providedSchema.size()) {
 
         // Is a provided schema with enough columns. Just
-        // use it.
-
-        outputSchema = providedSchema;
+        // use it. Do not add implicit cols to the table schema,
+        // but do add them to the reader schema.
+        tableSchema = providedSchema;
+        addImplicitCols(tableSchema);
+        readerSchema = defineReaderSchema(tableSchema);
       } else {
 
         // Have a provided schema, but more groups than
         // provided columns. Make up additional columns.
-
-        outputSchema = defineOutputSchemaFromProvided(providedSchema, groupCount);
+        tableSchema = defineTableSchemaFromProvided(providedSchema, groupCount);
+        readerSchema = defineReaderSchema(tableSchema);
       }
-      defineReaderSchema(schemaBuilder, outputSchema);
-
-      // Use the file framework to enable support for implicit and partition
-      // columns.
-
-      builder = new FileScanBuilder();
-      initScanBuilder(builder, scan);
-      builder.typeConverterBuilder().providedSchema(outputSchema);
     } else {
 
       // No schema provided;  use the columns framework to use the columns[] array
       // Also supports implicit and partition metadata.
-
-      schemaBuilder.addArray(ColumnsScanFramework.COLUMNS_COL, MinorType.VARCHAR);
-      ColumnsScanBuilder colScanBuilder = new ColumnsScanBuilder();
-      initScanBuilder(colScanBuilder, scan);
-      colScanBuilder.requireColumnsArray(true);
-      colScanBuilder.allowOtherCols(true);
-      builder = colScanBuilder;
+      readerSchema = buildArraySchema();
+      tableSchema = readerSchema;
     }
 
+    // Projection is based on our table schema which may or may not be
+    // the same as the provided schema.
+    builder.providedSchema(tableSchema);
+
     // Pass along the class that will create a batch reader on demand for
     // each input file.
-
     builder.setReaderFactory(new LogReaderFactory(
-        new LogReaderConfig(this, pattern, buildSchema(schemaBuilder),
-            !hasSchema, groupCount, maxErrors(providedSchema))));
+        new LogReaderConfig(this, pattern, providedSchema, tableSchema,
+            readerSchema, !hasSchema, groupCount, maxErrors(providedSchema))));
 
     // The default type of regex columns is nullable VarChar,
     // so let's use that as the missing column type.
-
-    builder.setNullType(Types.optional(MinorType.VARCHAR));
+    builder.nullType(Types.optional(MinorType.VARCHAR));
     return builder;
   }
 
@@ -234,7 +226,6 @@ public class LogFormatPlugin extends EasyFormatPlugin<LogFormatConfig> {
    * in. This schema drives type conversions if no schema is provided
    * for the table
    */
-
   private TupleMetadata defineOutputSchemaFromConfig(int capturingGroups) {
     List<String> fields = formatConfig.getFieldNames();
     for (int i = fields.size(); i < capturingGroups; i++) {
@@ -247,7 +238,6 @@ public class LogFormatPlugin extends EasyFormatPlugin<LogFormatConfig> {
     TupleMetadata schema = builder.buildSchema();
 
     // Populate the date formats, if provided.
-
     if (formatConfig.getSchema() == null) {
       return schema;
     }
@@ -267,6 +257,7 @@ public class LogFormatPlugin extends EasyFormatPlugin<LogFormatConfig> {
       }
       col.setProperty(ColumnMetadata.FORMAT_PROP, format);
     }
+    addImplicitCols(schema);
     return schema;
   }
 
@@ -281,8 +272,7 @@ public class LogFormatPlugin extends EasyFormatPlugin<LogFormatConfig> {
    * @return an output schema with the provided columns, plus default
    * columns for any missing columns
    */
-
-  private TupleMetadata defineOutputSchemaFromProvided(
+  private TupleMetadata defineTableSchemaFromProvided(
       TupleMetadata providedSchema, int capturingGroups) {
     assert capturingGroups >= providedSchema.size();
     SchemaBuilder builder = new SchemaBuilder();
@@ -292,7 +282,12 @@ public class LogFormatPlugin extends EasyFormatPlugin<LogFormatConfig> {
     for (int i = providedSchema.size(); i < capturingGroups; i++) {
       builder.addNullable("field_" + i, MinorType.VARCHAR);
     }
-    return builder.buildSchema();
+    TupleMetadata schema = builder.buildSchema();
+    if (providedSchema.hasProperties()) {
+      schema.properties().putAll(providedSchema.properties());
+    }
+    addImplicitCols(schema);
+    return schema;
   }
 
   /**
@@ -308,26 +303,37 @@ public class LogFormatPlugin extends EasyFormatPlugin<LogFormatConfig> {
    * to project, then will take the desired output type from the output
    * schema, providing any conversions needed
    */
-
-  private void defineReaderSchema(SchemaBuilder builder, TupleMetadata outputSchema) {
+  private TupleMetadata defineReaderSchema(TupleMetadata outputSchema) {
+    SchemaBuilder schemaBuilder = new SchemaBuilder();
     for (int i = 0; i < outputSchema.size(); i++) {
-      builder.addNullable(outputSchema.metadata(i).name(), MinorType.VARCHAR);
+      schemaBuilder.addNullable(outputSchema.metadata(i).name(), MinorType.VARCHAR);
     }
+    return schemaBuilder.build();
   }
 
-  private TupleMetadata buildSchema(SchemaBuilder builder) {
-    builder.addNullable(LogBatchReader.RAW_LINE_COL_NAME, MinorType.VARCHAR);
-    builder.addNullable(LogBatchReader.UNMATCHED_LINE_COL_NAME, MinorType.VARCHAR);
-    TupleMetadata schema = builder.buildSchema();
+  private TupleMetadata buildArraySchema() {
+    SchemaBuilder schemaBuilder = new SchemaBuilder();
+    schemaBuilder.addArray(ColumnsScanFramework.COLUMNS_COL, MinorType.VARCHAR);
+    TupleMetadata schema = schemaBuilder.build();
+    addImplicitCols(schema);
+    return schema;
+  }
 
-    // Exclude special columns from wildcard expansion
+  private void addImplicitCols(TupleMetadata schema) {
+    if (schema.index(LogBatchReader.RAW_LINE_COL_NAME) == -1) {
+      schema.addColumn(MetadataUtils.newScalar(LogBatchReader.RAW_LINE_COL_NAME,
+          Types.optional(MinorType.VARCHAR)));
+    }
+    if (schema.index(LogBatchReader.UNMATCHED_LINE_COL_NAME) == -1) {
+      schema.addColumn(MetadataUtils.newScalar(LogBatchReader.UNMATCHED_LINE_COL_NAME,
+          Types.optional(MinorType.VARCHAR)));
+    }
 
+    // Exclude special columns from wildcard expansion
     schema.metadata(LogBatchReader.RAW_LINE_COL_NAME).setBooleanProperty(
         ColumnMetadata.EXCLUDE_FROM_WILDCARD, true);
     schema.metadata(LogBatchReader.UNMATCHED_LINE_COL_NAME).setBooleanProperty(
         ColumnMetadata.EXCLUDE_FROM_WILDCARD, true);
-
-    return schema;
   }
 
   /**
@@ -336,7 +342,6 @@ public class LogFormatPlugin extends EasyFormatPlugin<LogFormatConfig> {
    * the plugin config. Then compile the regex. Issue an error if the
    * pattern is bad.
    */
-
   private Pattern setupPattern(TupleMetadata providedSchema) {
     String regex = formatConfig.getRegex();
     if (providedSchema != null) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ExtendedMockBatchReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ExtendedMockBatchReader.java
index 1fd7977..4edaddb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ExtendedMockBatchReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ExtendedMockBatchReader.java
@@ -95,7 +95,7 @@ public class ExtendedMockBatchReader implements ManagedReader<SchemaNegotiator>
                                           col.getConfig().getMajorType());
       schema.add(field);
     }
-    schemaNegotiator.setTableSchema(schema, true);
+    schemaNegotiator.tableSchema(schema, true);
 
     // Set the batch size. Ideally, we'd leave that to the framework based
     // on the bytes per batch. But, several legacy tests depend on a known,
@@ -106,7 +106,7 @@ public class ExtendedMockBatchReader implements ManagedReader<SchemaNegotiator>
 
     final int batchSize = config.getBatchSize();
     if (batchSize > 0) {
-      schemaNegotiator.setBatchSize(batchSize);
+      schemaNegotiator.batchSize(batchSize);
     }
 
     loader = schemaNegotiator.build();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockScanBatchCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockScanBatchCreator.java
index cfd3a89..5bcfeaa 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockScanBatchCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockScanBatchCreator.java
@@ -82,8 +82,8 @@ public class MockScanBatchCreator implements BatchCreator<MockSubScanPOP> {
     // each reader to adjust the batch size smaller if desired.
 
     ScanFrameworkBuilder builder = new ScanFrameworkBuilder();
-    builder.setBatchByteLimit(batchSizeBytes);
-    builder.setProjection(projList);
+    builder.batchByteLimit(batchSizeBytes);
+    builder.projection(projList);
     builder.setReaderFactory(new BasicScanFactory(readers.iterator()));
     ManagedScanFramework framework = new ManagedScanFramework(builder);
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapBatchReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapBatchReader.java
index 4aeff2f..b28880c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapBatchReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapBatchReader.java
@@ -119,7 +119,7 @@ public class PcapBatchReader implements ManagedReader<FileSchemaNegotiator> {
 
   private ScalarWriter isCorruptWriter;
 
-  private PcapReaderConfig readerConfig;
+  private final PcapReaderConfig readerConfig;
 
 
   // Writers for TCP Sessions
@@ -155,7 +155,7 @@ public class PcapBatchReader implements ManagedReader<FileSchemaNegotiator> {
 
     public boolean sessionizeTCPStreams;
 
-    private PcapFormatConfig config;
+    private final PcapFormatConfig config;
 
     public PcapReaderConfig(PcapFormatPlugin plugin) {
       this.plugin = plugin;
@@ -178,7 +178,7 @@ public class PcapBatchReader implements ManagedReader<FileSchemaNegotiator> {
     SchemaBuilder builder = new SchemaBuilder();
     Schema pcapSchema = new Schema(readerConfig.sessionizeTCPStreams);
     TupleMetadata schema = pcapSchema.buildSchema(builder);
-    negotiator.setTableSchema(schema, false);
+    negotiator.tableSchema(schema, false);
     ResultSetLoader loader = negotiator.build();
 
     // Creates writers for all fields (Since schema is known)
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapFormatPlugin.java
index b8e0175..40a70bd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapFormatPlugin.java
@@ -81,7 +81,7 @@ public class PcapFormatPlugin extends EasyFormatPlugin<PcapFormatConfig> {
     FileScanBuilder builder = new FileScanBuilder();
     builder.setReaderFactory(new PcapReaderFactory(new PcapReaderConfig(this)));
     initScanBuilder(builder, scan);
-    builder.setNullType(Types.optional(TypeProtos.MinorType.VARCHAR));
+    builder.nullType(Types.optional(TypeProtos.MinorType.VARCHAR));
     return builder;
   }
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/BaseScanOperatorExecTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/BaseScanOperatorExecTest.java
index 39c7a0f..5633943 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/BaseScanOperatorExecTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/BaseScanOperatorExecTest.java
@@ -104,7 +104,7 @@ public class BaseScanOperatorExecTest extends SubOperatorTest {
           .add("a", MinorType.INT)
           .addNullable("b", MinorType.VARCHAR, 10)
           .buildSchema();
-      schemaNegotiator.setTableSchema(schema, true);
+      schemaNegotiator.tableSchema(schema, true);
       tableLoader = schemaNegotiator.build();
       return true;
     }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/ScanTestUtils.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/ScanTestUtils.java
index b882b29..e0ebd0b 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/ScanTestUtils.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/ScanTestUtils.java
@@ -26,7 +26,7 @@ import org.apache.drill.exec.ops.OperatorContext;
 import org.apache.drill.exec.physical.base.AbstractSubScan;
 import org.apache.drill.exec.physical.base.Scan;
 import org.apache.drill.exec.physical.impl.scan.file.FileMetadataColumnDefn;
-import org.apache.drill.exec.physical.impl.scan.file.FileMetadataManager;
+import org.apache.drill.exec.physical.impl.scan.file.ImplicitColumnManager;
 import org.apache.drill.exec.physical.impl.scan.file.PartitionColumn;
 import org.apache.drill.exec.physical.impl.scan.framework.ManagedScanFramework.ScanFrameworkBuilder;
 import org.apache.drill.exec.physical.impl.scan.project.ReaderLevelProjection.ReaderProjectionResolver;
@@ -34,7 +34,7 @@ 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.ScanLevelProjection.ScanProjectionParser;
 import org.apache.drill.exec.physical.impl.scan.project.ScanSchemaOrchestrator.ScanOrchestratorBuilder;
-import org.apache.drill.exec.physical.resultSet.impl.RowSetTestUtils;
+import org.apache.drill.exec.physical.rowSet.RowSetTestUtils;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
@@ -66,19 +66,19 @@ public class ScanTestUtils {
     public abstract ScanFrameworkBuilder builder();
 
     public void projectAll() {
-      builder().setProjection(RowSetTestUtils.projectAll());
+      builder().projection(RowSetTestUtils.projectAll());
     }
 
     public void projectAllWithMetadata(int dirs) {
-      builder().setProjection(ScanTestUtils.projectAllWithMetadata(dirs));
+      builder().projection(ScanTestUtils.projectAllWithMetadata(dirs));
     }
 
     public void setProjection(String... projCols) {
-      builder().setProjection(RowSetTestUtils.projectList(projCols));
+      builder().projection(RowSetTestUtils.projectList(projCols));
     }
 
     public void setProjection(List<SchemaPath> projection) {
-      builder().setProjection(projection);
+      builder().projection(projection);
     }
 
     public ScanFixture build() {
@@ -150,7 +150,7 @@ public class ScanTestUtils {
    * @return schema with the metadata columns appended to the table columns
    */
 
-  public static TupleMetadata expandMetadata(TupleMetadata base, FileMetadataManager metadataProj, int dirCount) {
+  public static TupleMetadata expandMetadata(TupleMetadata base, ImplicitColumnManager metadataProj, int dirCount) {
     TupleMetadata metadataSchema = new TupleSchema();
     for (ColumnMetadata col : base) {
       metadataSchema.addColumn(col);
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 74f20e3..6009ed3 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
@@ -29,17 +29,17 @@ import org.apache.drill.exec.physical.impl.scan.ScanTestUtils.MockScanBuilder;
 import org.apache.drill.exec.physical.impl.scan.columns.ColumnsArrayManager;
 import org.apache.drill.exec.physical.impl.scan.columns.ColumnsScanFramework.ColumnsScanBuilder;
 import org.apache.drill.exec.physical.impl.scan.columns.ColumnsScanFramework;
-import org.apache.drill.exec.physical.impl.scan.file.FileMetadataManager;
-import org.apache.drill.exec.physical.impl.scan.file.FileMetadataManager.FileMetadataOptions;
+import org.apache.drill.exec.physical.impl.scan.file.ImplicitColumnManager;
+import org.apache.drill.exec.physical.impl.scan.file.ImplicitColumnManager.ImplicitColumnOptions;
 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.ScanOrchestratorBuilder;
 import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
-import org.apache.drill.exec.physical.resultSet.impl.RowSetTestUtils;
 import org.apache.drill.exec.record.metadata.SchemaBuilder;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
 import org.apache.drill.test.SubOperatorTest;
+import org.apache.drill.exec.physical.rowSet.RowSetTestUtils;
 import org.apache.drill.exec.physical.rowSet.RowSet.SingleRowSet;
 import org.apache.drill.test.rowSet.RowSetUtilities;
 import org.apache.hadoop.fs.Path;
@@ -59,8 +59,8 @@ public class TestColumnsArray extends SubOperatorTest {
     ResultSetLoader loader;
   }
 
-  private FileMetadataOptions standardOptions(Path filePath) {
-    FileMetadataOptions options = new FileMetadataOptions();
+  private ImplicitColumnOptions standardOptions(Path filePath) {
+    ImplicitColumnOptions options = new ImplicitColumnOptions();
     options.useLegacyWildcardExpansion(false); // Don't expand partition columns for wildcard
     options.setSelectionRoot(new Path("hdfs:///w"));
     options.setFiles(Lists.newArrayList(filePath));
@@ -74,7 +74,7 @@ public class TestColumnsArray extends SubOperatorTest {
     // Set up the file metadata manager
 
     Path filePath = new Path("hdfs:///w/x/y/z.csv");
-    FileMetadataManager metadataManager = new FileMetadataManager(
+    ImplicitColumnManager metadataManager = new ImplicitColumnManager(
         fixture.getOptionManager(),
         standardOptions(filePath));
 
@@ -85,13 +85,13 @@ public class TestColumnsArray extends SubOperatorTest {
     // Configure the schema orchestrator
 
     ScanOrchestratorBuilder builder = new MockScanBuilder();
-    builder.withMetadata(metadataManager);
+    builder.withImplicitColumns(metadataManager);
     builder.addParser(colsManager.projectionParser());
     builder.addResolver(colsManager.resolver());
 
     // SELECT <proj list> ...
 
-    builder.setProjection(projList);
+    builder.projection(projList);
     mock.scanner = new ScanSchemaOrchestrator(fixture.allocator(), builder);
 
     // FROM z.csv
@@ -264,7 +264,7 @@ public class TestColumnsArray extends SubOperatorTest {
     ScanOrchestratorBuilder builder = new ColumnsScanBuilder();
     builder.addParser(colsManager.projectionParser());
     builder.addResolver(colsManager.resolver());
-    builder.setProjection(cols);
+    builder.projection(cols);
     return new ScanSchemaOrchestrator(fixture.allocator(), builder);
... 8918 lines suppressed ...