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 2019/03/29 14:47:45 UTC

[drill] 02/03: DRILL-7011: Support schema in 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 d89d5fc769557aef4c74c0b33b9c8f854035da93
Author: Paul Rogers <pr...@cloudera.com>
AuthorDate: Sun Mar 17 13:57:54 2019 -0700

    DRILL-7011: Support schema in scan framework
    
    * Adds schema support to the row set-based scan framework and to the "V3" text reader based on that framework.
    * Adding the schema made clear that passing options as a long list of constructor arguments was not sustainable. Refactored code to use a builder pattern instead.
    * Added support for default values in the "null column loader", which required adding a "setValue" method to the column accessors.
    * Added unit tests for all new or changed functionality. See TestCsvWithSchema for the overall test of the entire integrated mechanism.
    * Added tests for explicit projection with schema
    * Better handling of date/time in column accessors
    * Converted recent column metadata work from Java 8 date/time to Joda.
    * Added more CSV-with-schema unit tests
    * Removed the ID fields from "resolved columns", used "instanceof" instead.
    * Added wildcard projection with an output schema. Handles both "lenient" and "strict" schemas.
    * Tagged projection columns with their output schema, when available.
    * Scan projection added modes for wildcard with an output schema. The reader projection added support for merging reader and output schemas.
    * Includes refactoring of scan operator tests (the test file grew too large.)
    * Renamed some classes to avoid confusing reader schemas with output schemas.
    * Added unit tests for the new functionality.
    * Added "lenient" wildcard with schema test for CSV
    * Added more type conversions: string-to-bit, many-to-string
    * Fixed bug in column writer for VarDecimal
    * Added missing unit tests, and fixed bugs, in Bit column reader/writer
    * Cleaned up a number of unneded "SuppressWarnings"
    
    closes #1711
---
 .../drill/common/exceptions/UserException.java     |   37 +-
 .../exec/cache/VectorAccessibleSerializable.java   |    4 -
 .../apache/drill/exec/cache/VectorSerializer.java  |    1 -
 .../drill/exec/client/LoggingResultsListener.java  |    1 -
 .../apache/drill/exec/compile/ClassBuilder.java    |    1 -
 .../drill/exec/compile/ClassTransformer.java       |    1 -
 .../expr/fn/FunctionImplementationRegistry.java    |    2 -
 .../exec/physical/impl/MergingReceiverCreator.java |    1 -
 .../apache/drill/exec/physical/impl/ScanBatch.java |    2 +-
 .../physical/impl/TopN/PriorityQueueTemplate.java  |    5 +-
 .../drill/exec/physical/impl/TopN/TopNBatch.java   |    7 -
 .../exec/physical/impl/WriterRecordBatch.java      |    2 -
 .../exec/physical/impl/aggregate/HashAggBatch.java |    5 +-
 .../physical/impl/aggregate/HashAggTemplate.java   |    4 +-
 .../physical/impl/aggregate/StreamingAggBatch.java |    3 -
 .../physical/impl/common/ChainedHashTable.java     |    1 -
 .../physical/impl/common/HashTableTemplate.java    |    8 +-
 .../physical/impl/flatten/FlattenRecordBatch.java  |    4 -
 .../impl/mergereceiver/MergingRecordBatch.java     |    3 -
 .../OrderedPartitionRecordBatch.java               |    5 -
 .../OrderedPartitionSenderCreator.java             |    1 -
 .../impl/partitionsender/PartitionerTemplate.java  |    1 -
 .../impl/scan/columns/ColumnsArrayManager.java     |    8 +-
 .../impl/scan/columns/ColumnsArrayParser.java      |    4 +-
 .../impl/scan/columns/ColumnsScanFramework.java    |   74 +-
 .../impl/scan/columns/ColumnsSchemaNegotiator.java |    5 +-
 .../scan/columns/ResolvedColumnsArrayColumn.java   |    5 -
 .../scan/columns/UnresolvedColumnsArrayColumn.java |    8 +-
 .../impl/scan/file/BaseFileScanFramework.java      |  176 ---
 .../impl/scan/file/FileMetadataColumn.java         |    7 +-
 .../impl/scan/file/FileMetadataColumnsParser.java  |    4 +-
 .../impl/scan/file/FileMetadataManager.java        |  135 +-
 .../physical/impl/scan/file/FileScanFramework.java |  189 ++-
 .../physical/impl/scan/file/MetadataColumn.java    |    8 +-
 .../physical/impl/scan/file/PartitionColumn.java   |    5 -
 ...sicScanFramework.java => BasicScanFactory.java} |   34 +-
 .../impl/scan/framework/ManagedReader.java         |    2 +-
 ...canFramework.java => ManagedScanFramework.java} |  127 +-
 .../impl/scan/framework/SchemaNegotiatorImpl.java  |   23 +-
 .../impl/scan/framework/ShimBatchReader.java       |   22 +-
 ...edColumn.java => AbstractUnresolvedColumn.java} |   39 +-
 .../impl/scan/project/ColumnProjection.java        |   29 -
 .../scan/project/ExplicitSchemaProjection.java     |   23 +-
 .../impl/scan/project/MetadataManager.java         |    4 +-
 .../impl/scan/project/NoOpMetadataManager.java     |    4 +-
 .../impl/scan/project/NullColumnBuilder.java       |   96 +-
 .../impl/scan/project/NullColumnLoader.java        |   47 +-
 ...lProjection.java => ReaderLevelProjection.java} |   20 +-
 .../scan/project/ReaderSchemaOrchestrator.java     |   66 +-
 .../physical/impl/scan/project/ResolvedColumn.java |   15 +-
 .../impl/scan/project/ResolvedMapColumn.java       |    5 -
 .../impl/scan/project/ResolvedNullColumn.java      |   28 +-
 .../impl/scan/project/ResolvedTableColumn.java     |   17 +-
 .../physical/impl/scan/project/ResolvedTuple.java  |    5 +-
 .../impl/scan/project/ScanLevelProjection.java     |  203 ++-
 .../impl/scan/project/ScanSchemaOrchestrator.java  |  262 ++--
 .../physical/impl/scan/project/SchemaSmoother.java |   10 +-
 .../impl/scan/project/SmoothingProjection.java     |   20 +-
 ...hemaProjection.java => WildcardProjection.java} |   11 +-
 .../scan/project/WildcardSchemaProjection.java     |   81 +-
 .../physical/impl/sort/SortRecordBatchBuilder.java |    2 -
 .../drill/exec/physical/impl/spill/SpillSet.java   |    4 +-
 .../physical/impl/svremover/GenericSV4Copier.java  |    1 -
 .../physical/impl/union/UnionAllRecordBatch.java   |    2 +-
 .../physical/impl/unnest/UnnestRecordBatch.java    |    4 +-
 .../UnorderedReceiverCreator.java                  |    1 -
 .../physical/impl/validate/BatchValidator.java     |    4 +-
 .../physical/impl/xsort/ExternalSortBatch.java     |    8 +-
 .../exec/physical/impl/xsort/MSortTemplate.java    |    2 -
 .../impl/xsort/managed/BufferedBatches.java        |    3 -
 .../impl/xsort/managed/ExternalSortBatch.java      |    1 -
 .../physical/impl/xsort/managed/MSortTemplate.java |    2 -
 .../xsort/managed/PriorityQueueCopierTemplate.java |    1 -
 .../xsort/managed/PriorityQueueCopierWrapper.java  |    1 -
 .../exec/physical/rowSet/impl/BuildFromSchema.java |    1 -
 .../rowSet/impl/DefaultSchemaTransformer.java      |    4 +
 .../physical/rowSet/impl/SchemaTransformer.java    |    2 +
 .../rowSet/impl/SchemaTransformerImpl.java         |    5 +-
 .../exec/physical/rowSet/impl/TupleState.java      |   19 +-
 .../rowSet/project/RequestedColumnImpl.java        |   14 +-
 .../drill/exec/record/HyperVectorWrapper.java      |    1 -
 .../drill/exec/record/RecordBatchLoader.java       |    1 -
 .../apache/drill/exec/record/RecordBatchSizer.java |    7 +-
 .../org/apache/drill/exec/record/SchemaUtil.java   |    3 -
 .../drill/exec/record/SimpleVectorWrapper.java     |    1 -
 .../apache/drill/exec/record/WritableBatch.java    |    4 -
 .../record/metadata/AbstractColumnMetadata.java    |    4 +-
 .../exec/record/metadata/MapColumnMetadata.java    |    1 -
 .../record/metadata/PrimitiveColumnMetadata.java   |   39 +-
 .../drill/exec/record/metadata/SchemaBuilder.java  |   10 +
 .../drill/exec/record/metadata/TupleSchema.java    |    4 +
 .../org/apache/drill/exec/server/Drillbit.java     |    3 -
 .../apache/drill/exec/server/RemoteServiceSet.java |    1 -
 .../drill/exec/server/rest/DrillRestServer.java    |    2 -
 .../apache/drill/exec/server/rest/DrillRoot.java   |    8 -
 .../drill/exec/server/rest/StatusResources.java    |    2 -
 .../drill/exec/server/rest/StorageResources.java   |    1 -
 .../apache/drill/exec/server/rest/WebServer.java   |    2 -
 .../exec/server/rest/profile/ProfileResources.java |    3 -
 .../exec/store/dfs/DrillFSDataInputStream.java     |    1 -
 .../exec/store/dfs/FileSystemSchemaFactory.java    |    1 -
 .../exec/store/dfs/easy/EasyFormatPlugin.java      |   27 +-
 .../exec/store/easy/text/TextFormatPlugin.java     |   69 +-
 .../easy/text/compliant/RepeatedVarCharOutput.java |    3 -
 .../columnreaders/FixedWidthRepeatedReader.java    |    1 -
 .../columnreaders/ParquetColumnMetadata.java       |    1 -
 .../store/parquet2/DrillParquetGroupConverter.java |    2 -
 .../drill/exec/store/sys/FunctionsIterator.java    |    1 -
 .../drill/exec/util/StoragePluginTestUtils.java    |    1 -
 .../drill/exec/vector/complex/fn/JsonReader.java   |    4 -
 .../drill/exec/vector/complex/fn/VectorOutput.java |    4 -
 .../vector/complex/impl/VectorContainerWriter.java |    1 -
 .../drill/exec/work/foreman/FragmentsRunner.java   |    1 -
 .../drill/exec/work/foreman/QueryManager.java      |    2 -
 .../work/foreman/rm/DistributedQueryQueue.java     |    1 -
 .../work/foreman/rm/DynamicResourceManager.java    |    1 -
 .../work/foreman/rm/ResourceManagerBuilder.java    |    1 -
 .../drill/exec/work/fragment/FragmentExecutor.java |    5 +-
 .../apache/drill/exec/work/user/PlanSplitter.java  |    1 -
 .../test/java/org/apache/drill/PlanningBase.java   |    1 -
 .../java/org/apache/drill/TestSchemaCommands.java  |    2 +-
 .../apache/drill/exec/cache/TestWriteToDisk.java   |    1 -
 .../exec/fn/interp/ExpressionInterpreterTest.java  |    3 -
 .../exec/physical/impl/TestConvertFunctions.java   |    2 -
 .../exec/physical/impl/TestSimpleFunctions.java    |    1 -
 .../impl/mergereceiver/TestMergingReceiver.java    |    7 -
 .../impl/scan/BaseScanOperatorExecTest.java        |  180 +++
 .../exec/physical/impl/scan/ScanTestUtils.java     |   74 +-
 .../exec/physical/impl/scan/TestColumnsArray.java  |   38 +-
 .../impl/scan/TestColumnsArrayFramework.java       |   96 +-
 .../physical/impl/scan/TestColumnsArrayParser.java |   33 +-
 .../impl/scan/TestFileMetadataColumnParser.java    |  197 ++-
 .../impl/scan/TestFileMetadataProjection.java      |   50 +-
 .../physical/impl/scan/TestFileScanFramework.java  |  166 +-
 .../physical/impl/scan/TestScanOperExecBasics.java |  398 +++++
 .../impl/scan/TestScanOperExecEarlySchema.java     |  260 ++++
 .../impl/scan/TestScanOperExecLateSchema.java      |  402 +++++
 .../impl/scan/TestScanOperExecOuputSchema.java     |  253 +++
 .../impl/scan/TestScanOperExecOverflow.java        |  195 +++
 .../impl/scan/TestScanOperExecSmoothing.java       |  257 ++++
 .../physical/impl/scan/TestScanOperatorExec.java   | 1603 --------------------
 .../impl/scan/TestScanOrchestratorEarlySchema.java |   94 +-
 .../impl/scan/TestScanOrchestratorLateSchema.java  |   11 +-
 .../impl/scan/TestScanOrchestratorMetadata.java    |   83 +-
 .../impl/scan/project/TestNullColumnLoader.java    |  221 ++-
 ...jection.java => TestReaderLevelProjection.java} |  166 +-
 .../impl/scan/project/TestRowBatchMerger.java      |   12 +-
 .../impl/scan/project/TestScanLevelProjection.java |   90 +-
 .../impl/scan/project/TestSchemaSmoothing.java     |   72 +-
 .../physical/impl/writer/TestParquetWriter.java    |    1 -
 .../impl/xsort/TestSimpleExternalSort.java         |   13 +-
 .../impl/xsort/managed/SortTestUtilities.java      |    9 +-
 .../physical/impl/xsort/managed/TestCopier.java    |   31 +-
 .../impl/xsort/managed/TestShortArrays.java        |    1 -
 .../physical/impl/xsort/managed/TestSortImpl.java  |   13 +-
 .../physical/impl/xsort/managed/TestSorter.java    |   13 +-
 .../exec/physical/unit/MiniPlanUnitTestBase.java   |    1 -
 .../metadata/schema/parser/TestSchemaParser.java   |    4 +-
 .../apache/drill/exec/record/vector/TestLoad.java  |    3 -
 .../drill/exec/record/vector/TestValueVector.java  |    6 +-
 .../drill/exec/rpc/data/TestBitBitKerberos.java    |    1 -
 .../drill/exec/server/TestDrillbitResilience.java  |    2 -
 .../store/easy/text/compliant/BaseCsvTest.java     |   13 +
 .../easy/text/compliant/TestCsvWithSchema.java     |  793 ++++++++++
 .../easy/text/compliant/TestPartitionRace.java     |   13 +-
 .../columnreaders/TestBatchSizingMemoryUtil.java   |    1 -
 .../java/org/apache/drill/test/BaseTestQuery.java  |    1 -
 .../java/org/apache/drill/test/ClusterFixture.java |    4 -
 .../drill/test/ClusterMockStorageFixture.java      |    1 -
 .../drill/test/LegacyOperatorTestBuilder.java      |    2 +-
 .../org/apache/drill/test/OperatorFixture.java     |    2 -
 .../org/apache/drill/test/OperatorTestBuilder.java |    2 +-
 .../java/org/apache/drill/test/QueryBuilder.java   |    1 -
 .../java/org/apache/drill/test/QueryTestUtil.java  |    3 -
 .../apache/drill/test/rowSet/HyperRowSetImpl.java  |    1 -
 .../apache/drill/test/rowSet/RowSetComparison.java |    5 +-
 .../test/rowSet/test/TestColumnConverter.java      |  274 +++-
 .../drill/test/rowSet/test/TestFillEmpties.java    |    3 +
 .../test/rowSet/test/TestIndirectReaders.java      |    2 -
 .../test/rowSet/test/TestScalarAccessors.java      |  346 +++--
 .../test/rowSet/test/TestVariantAccessors.java     |    2 -
 .../org/apache/drill/vector/TestFillEmpties.java   |    4 -
 .../org/apache/drill/vector/TestToNullable.java    |    6 +-
 .../main/codegen/templates/ColumnAccessors.java    |  183 ++-
 .../drill/exec/record/MaterializedField.java       |   70 +-
 .../exec/record/metadata/AbstractPropertied.java   |   12 +-
 .../drill/exec/record/metadata/ColumnMetadata.java |    3 +-
 .../drill/exec/record/metadata/ProjectionType.java |   60 +-
 .../drill/exec/record/metadata/Propertied.java     |    2 +
 .../drill/exec/record/metadata/TupleMetadata.java  |    2 +
 .../drill/exec/vector/accessor/ScalarReader.java   |   25 +
 .../drill/exec/vector/accessor/ScalarWriter.java   |   15 +
 .../drill/exec/vector/accessor/ValueType.java      |   21 +
 ...gToLong.java => AbstractConvertFromString.java} |   27 +-
 .../accessor/convert/AbstractWriteConverter.java   |    2 +-
 ...ckage-info.java => ConvertBooleanToString.java} |   28 +-
 ...tStringToTime.java => ConvertDateToString.java} |   38 +-
 ...ringToLong.java => ConvertDecimalToString.java} |   29 +-
 ...tringToLong.java => ConvertDoubleToString.java} |   27 +-
 ...rtStringToLong.java => ConvertIntToString.java} |   27 +-
 ...ingToLong.java => ConvertIntervalToString.java} |   30 +-
 ...tStringToLong.java => ConvertLongToString.java} |   27 +-
 ...ngToDouble.java => ConvertStringToBoolean.java} |   13 +-
 .../accessor/convert/ConvertStringToDate.java      |   17 +-
 ...tringToInt.java => ConvertStringToDecimal.java} |   15 +-
 .../accessor/convert/ConvertStringToDouble.java    |    4 +-
 .../accessor/convert/ConvertStringToInt.java       |    4 +-
 .../accessor/convert/ConvertStringToInterval.java  |    4 +-
 .../accessor/convert/ConvertStringToLong.java      |    4 +-
 .../accessor/convert/ConvertStringToTime.java      |   16 +-
 .../accessor/convert/ConvertStringToTimeStamp.java |   15 +-
 ...imeStamp.java => ConvertTimeStampToString.java} |   37 +-
 ...tStringToTime.java => ConvertTimeToString.java} |   38 +-
 .../accessor/convert/StandardConversions.java      |  104 +-
 .../exec/vector/accessor/convert/package-info.java |   23 +-
 .../accessor/reader/AbstractScalarReader.java      |   33 +-
 .../accessor/writer/AbstractScalarWriter.java      |    2 +
 .../accessor/writer/NullableScalarWriter.java      |   11 +
 .../accessor/writer/OffsetVectorWriterImpl.java    |    6 +
 .../accessor/writer/dummy/DummyScalarWriter.java   |    3 +
 220 files changed, 6171 insertions(+), 3577 deletions(-)

diff --git a/common/src/main/java/org/apache/drill/common/exceptions/UserException.java b/common/src/main/java/org/apache/drill/common/exceptions/UserException.java
index 3fcdc07..2aeb5cd 100644
--- a/common/src/main/java/org/apache/drill/common/exceptions/UserException.java
+++ b/common/src/main/java/org/apache/drill/common/exceptions/UserException.java
@@ -17,17 +17,17 @@
  */
 package org.apache.drill.common.exceptions;
 
-import org.apache.drill.exec.proto.CoordinationProtos;
-import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.proto.UserBitShared.DrillPBError;
-import org.slf4j.Logger;
+import static java.lang.Thread.sleep;
 
-import java.io.File;
 import java.io.BufferedWriter;
+import java.io.File;
 import java.io.FileWriter;
 import java.lang.management.ManagementFactory;
 
-import static java.lang.Thread.sleep;
+import org.apache.drill.exec.proto.CoordinationProtos;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.UserBitShared.DrillPBError;
+import org.slf4j.Logger;
 /**
  * Base class for all user exception. The goal is to separate out common error conditions where we can give users
  * useful feedback.
@@ -61,7 +61,7 @@ public class UserException extends DrillRuntimeException {
   }
 
   public static Builder memoryError(final String format, final Object... args) {
-    Builder builder =  UserException.resourceError();
+    final Builder builder =  UserException.resourceError();
     builder.message(MEMORY_ERROR_MSG);
     if (!format.isEmpty()) {
       builder.addContext(String.format(format, args));
@@ -89,9 +89,6 @@ public class UserException extends DrillRuntimeException {
    * @param cause exception we want the user exception to wrap. If cause is, or wrap, a user exception it will be
    *              returned by the builder instead of creating a new user exception
    * @return user exception builder
-   *
-   * @deprecated This method should never need to be used explicitly, unless you are passing the exception to the
-   *             Rpc layer or UserResultListener.submitFailed()
    */
 
   public static Builder systemError(final Throwable cause) {
@@ -598,32 +595,32 @@ public class UserException extends DrillRuntimeException {
       //  release the spinning threads do "clush -a rm /tmp/drill/spin")
       // The processID of the spinning thread (along with the error message) should then be found
       // in a file like  /tmp/drill/spin4148663301172491613.tmp
-      File spinFile = new File("/tmp/drill/spin");
+      final File spinFile = new File("/tmp/drill/spin");
       if ( spinFile.exists() ) {
-        File tmpDir = new File("/tmp/drill");
+        final File tmpDir = new File("/tmp/drill");
         File outErr = null;
         try {
           outErr = File.createTempFile("spin", ".tmp", tmpDir);
-          BufferedWriter bw = new BufferedWriter(new FileWriter(outErr));
+          final BufferedWriter bw = new BufferedWriter(new FileWriter(outErr));
           bw.write("Spinning process: " + ManagementFactory.getRuntimeMXBean().getName()
           /* After upgrading to JDK 9 - replace with: ProcessHandle.current().getPid() */);
           bw.write("\nError cause: " +
             (errorType == DrillPBError.ErrorType.SYSTEM ? ("SYSTEM ERROR: " + ErrorHelper.getRootMessage(cause)) : message));
           bw.close();
-        } catch (Exception ex) {
+        } catch (final Exception ex) {
           logger.warn("Failed creating a spinner tmp message file: {}", ex);
         }
         while (spinFile.exists()) {
-          try { sleep(1_000); } catch (Exception ex) { /* ignore interruptions */ }
+          try { sleep(1_000); } catch (final Exception ex) { /* ignore interruptions */ }
         }
-        try { outErr.delete(); } catch (Exception ex) { } // cleanup - remove err msg file
+        try { outErr.delete(); } catch (final Exception ex) { } // cleanup - remove err msg file
       }
 
       if (uex != null) {
         return uex;
       }
 
-      boolean isSystemError = errorType == DrillPBError.ErrorType.SYSTEM;
+      final boolean isSystemError = errorType == DrillPBError.ErrorType.SYSTEM;
 
       // make sure system errors use the root error message and display the root cause class name
       if (isSystemError) {
@@ -639,7 +636,7 @@ public class UserException extends DrillRuntimeException {
       if (isSystemError) {
         logger.error(newException.getMessage(), newException);
       } else {
-        StringBuilder buf = new StringBuilder();
+        final StringBuilder buf = new StringBuilder();
         buf.append("User Error Occurred");
         if (message != null) {
           buf.append(": ").append(message);
@@ -758,7 +755,7 @@ public class UserException extends DrillRuntimeException {
   }
 
   public String getErrorLocation() {
-    DrillbitEndpoint ep = context.getEndpoint();
+    final DrillbitEndpoint ep = context.getEndpoint();
     if (ep != null) {
       return ep.getAddress() + ":" + ep.getUserPort();
     } else {
@@ -774,7 +771,7 @@ public class UserException extends DrillRuntimeException {
    * @return generated user error message
    */
   private String generateMessage(boolean includeErrorIdAndIdentity) {
-    boolean seeLogsMessage = errorType == DrillPBError.ErrorType.INTERNAL_ERROR
+    final boolean seeLogsMessage = errorType == DrillPBError.ErrorType.INTERNAL_ERROR
         || errorType == DrillPBError.ErrorType.SYSTEM;
     return errorType + " ERROR: " + super.getMessage() + "\n\n" +
         context.generateContextMessage(includeErrorIdAndIdentity, seeLogsMessage);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorAccessibleSerializable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorAccessibleSerializable.java
index 350e9c7..46fc72f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorAccessibleSerializable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorAccessibleSerializable.java
@@ -112,13 +112,11 @@ public class VectorAccessibleSerializable extends AbstractStreamSerializable {
     }
     final int dataLength = recordCount * SelectionVector2.RECORD_SIZE;
     svMode = BatchSchema.SelectionVectorMode.TWO_BYTE;
-    @SuppressWarnings("resource")
     DrillBuf buf = allocator.read(dataLength, input);
     sv2 = new SelectionVector2(allocator, buf, recordCount);
     buf.release(); // SV2 now owns the buffer
   }
 
-  @SuppressWarnings("resource")
   private void readVectors(InputStream input, RecordBatchDef batchDef) throws IOException {
     final VectorContainer container = new VectorContainer();
     final List<ValueVector> vectorList = Lists.newArrayList();
@@ -139,7 +137,6 @@ public class VectorAccessibleSerializable extends AbstractStreamSerializable {
   }
 
   // Like above, only preserve the original container and list of value-vectors
-  @SuppressWarnings("resource")
   public void readFromStreamWithContainer(VectorContainer myContainer, InputStream input) throws IOException {
     final VectorContainer container = new VectorContainer();
     final UserBitShared.RecordBatchDef batchDef = UserBitShared.RecordBatchDef.parseDelimitedFrom(input);
@@ -203,7 +200,6 @@ public class VectorAccessibleSerializable extends AbstractStreamSerializable {
    * @param output the OutputStream to write to
    * @throws IOException
    */
-  @SuppressWarnings("resource")
   @Override
   public void writeToStream(OutputStream output) throws IOException {
     Preconditions.checkNotNull(output);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorSerializer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorSerializer.java
index 9f0ab9f..2a18641 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorSerializer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorSerializer.java
@@ -72,7 +72,6 @@ public class VectorSerializer {
       return write(va, null);
     }
 
-    @SuppressWarnings("resource")
     public int write(VectorAccessible va, SelectionVector2 sv2) throws IOException {
       checkNotNull(va);
       WritableBatch batch = WritableBatch.getBatchNoHVWrap(
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/client/LoggingResultsListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/client/LoggingResultsListener.java
index f84ca95..951b33d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/client/LoggingResultsListener.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/client/LoggingResultsListener.java
@@ -69,7 +69,6 @@ public class LoggingResultsListener implements UserResultsListener {
   }
 
   @Override
-  @SuppressWarnings("resource")
   public void dataArrived(QueryDataBatch result, ConnectionThrottle throttle) {
     final QueryData header = result.getHeader();
     final DrillBuf data = result.getData();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassBuilder.java
index 075bcd3..b4790c8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassBuilder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassBuilder.java
@@ -129,7 +129,6 @@ public class ClassBuilder {
    * @throws ClassTransformationException generic "something is wrong" error from
    * Drill class compilation code.
    */
-  @SuppressWarnings("resource")
   private Class<?> compileClass(CodeGenerator<?> cg) throws IOException, CompileException, ClassNotFoundException, ClassTransformationException {
     final long t1 = System.nanoTime();
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
index 8248d73..e2ce922 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
@@ -217,7 +217,6 @@ public class ClassTransformer {
     }
   }
 
-  @SuppressWarnings("resource")
   public Class<?> getImplementationClass(CodeGenerator<?> cg) throws ClassTransformationException {
     final QueryClassLoader loader = new QueryClassLoader(config, optionManager);
     return getImplementationClass(loader, cg.getDefinition(),
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionImplementationRegistry.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionImplementationRegistry.java
index 1435605..30c194a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionImplementationRegistry.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionImplementationRegistry.java
@@ -347,7 +347,6 @@ public class FunctionImplementationRegistry implements FunctionLookupContext, Au
    * @param version remote function registry local function registry was based on
    * @return true if remote and local function registries were synchronized after given version
    */
-  @SuppressWarnings("resource")
   public boolean syncWithRemoteRegistry(int version) {
     // Do the version check only if a remote registry exists. It does
     // not exist for some JMockit-based unit tests.
@@ -561,7 +560,6 @@ public class FunctionImplementationRegistry implements FunctionLookupContext, Au
    * @return local path to jar that was copied
    * @throws IOException in case of problems during jar coping process
    */
-  @SuppressWarnings("resource")
   private Path copyJarToLocal(String jarName, RemoteFunctionRegistry remoteFunctionRegistry) throws IOException {
     Path registryArea = remoteFunctionRegistry.getRegistryArea();
     FileSystem fs = remoteFunctionRegistry.getFs();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/MergingReceiverCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/MergingReceiverCreator.java
index 66a0cc2..9d80f56 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/MergingReceiverCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/MergingReceiverCreator.java
@@ -31,7 +31,6 @@ import org.apache.drill.exec.work.batch.RawBatchBuffer;
 public class MergingReceiverCreator implements BatchCreator<MergingReceiverPOP> {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MergingReceiverCreator.class);
 
-  @SuppressWarnings("resource")
   @Override
   public MergingRecordBatch getBatch(ExecutorFragmentContext context,
                               MergingReceiverPOP receiver,
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
index 1abc3d8..07e2ae5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
@@ -367,6 +367,7 @@ public class ScanBatch implements CloseableRecordBatch {
     return container.getValueAccessorById(clazz, ids);
   }
 
+  @SuppressWarnings("unused")
   private void logRecordBatchStats() {
     final int MAX_FQN_LENGTH = 50;
 
@@ -552,7 +553,6 @@ public class ScanBatch implements CloseableRecordBatch {
     private void populateImplicitVectors(Map<String, String> implicitValues, int recordCount) {
       if (implicitValues != null) {
         for (Map.Entry<String, String> entry : implicitValues.entrySet()) {
-          @SuppressWarnings("resource")
           final NullableVarCharVector v = (NullableVarCharVector) implicitFieldVectorMap.get(entry.getKey());
           String val;
           if ((val = entry.getValue()) != null) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/PriorityQueueTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/PriorityQueueTemplate.java
index 54243da..9518165 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/PriorityQueueTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/PriorityQueueTemplate.java
@@ -61,7 +61,6 @@ public abstract class PriorityQueueTemplate implements PriorityQueue {
   public void init(int limit, BufferAllocator allocator,  boolean hasSv2) throws SchemaChangeException {
     this.limit = limit;
     this.allocator = allocator;
-    @SuppressWarnings("resource")
     // It's allocating memory to store (limit+1) indexes. When first limit number of record indexes are stored then all
     // the other record indexes are kept at (limit+1) and evaluated with the root element of heap to determine if
     // this new element will reside in heap or not.
@@ -85,7 +84,6 @@ public abstract class PriorityQueueTemplate implements PriorityQueue {
     cleanup();
     hyperBatch = new ExpandableHyperContainer(newContainer);
     batchCount = hyperBatch.iterator().next().getValueVectors().length;
-    @SuppressWarnings("resource")
     final DrillBuf drillBuf = allocator.buffer(4 * (limit + 1));
     heapSv4 = new SelectionVector4(drillBuf, limit, Character.MAX_VALUE);
     // Reset queue size (most likely to be set to limit).
@@ -98,7 +96,6 @@ public abstract class PriorityQueueTemplate implements PriorityQueue {
     doSetup(hyperBatch, null);
   }
 
-  @SuppressWarnings("resource")
   @Override
   public void add(RecordBatchData batch) throws SchemaChangeException{
     Stopwatch watch = Stopwatch.createStarted();
@@ -143,7 +140,6 @@ public abstract class PriorityQueueTemplate implements PriorityQueue {
   @Override
   public void generate() throws SchemaChangeException {
     Stopwatch watch = Stopwatch.createStarted();
-    @SuppressWarnings("resource")
     final DrillBuf drillBuf = allocator.buffer(4 * queueSize);
     finalSv4 = new SelectionVector4(drillBuf, queueSize, 4000);
     for (int i = queueSize - 1; i >= 0; i--) {
@@ -190,6 +186,7 @@ public abstract class PriorityQueueTemplate implements PriorityQueue {
    * @return - true - queue is still initialized
    *           false - queue is not yet initialized and before using queue init should be called
    */
+  @Override
   public boolean isInitialized() {
     return (heapSv4 != null);
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/TopNBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/TopNBatch.java
index aaca8a5..5ae6e76 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/TopNBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/TopNBatch.java
@@ -74,7 +74,6 @@ import static org.apache.drill.exec.record.RecordBatch.IterOutcome.EMIT;
 import static org.apache.drill.exec.record.RecordBatch.IterOutcome.NONE;
 import static org.apache.drill.exec.record.RecordBatch.IterOutcome.OK;
 import static org.apache.drill.exec.record.RecordBatch.IterOutcome.OK_NEW_SCHEMA;
-import static org.apache.drill.exec.record.RecordBatch.IterOutcome.STOP;
 
 /**
  * Operator Batch which implements the TopN functionality. It is more efficient than (sort + limit) since unlike sort
@@ -143,7 +142,6 @@ public class TopNBatch extends AbstractRecordBatch<TopN> {
       case OK:
       case OK_NEW_SCHEMA:
         for (VectorWrapper<?> w : incoming) {
-          @SuppressWarnings("resource")
           ValueVector v = container.addOrGet(w.getField());
           if (v instanceof AbstractContainerVector) {
             w.getValueVector().makeTransferPair(v);
@@ -344,7 +342,6 @@ public class TopNBatch extends AbstractRecordBatch<TopN> {
     // Simple VectorConatiner which stores limit number of records only. The records whose indexes are stored inside
     // selectionVector4 below are only copied from Hyper container to this simple container.
     VectorContainer newContainer = new VectorContainer(oContext);
-    @SuppressWarnings("resource")
     // SV4 storing the limit number of indexes
     SelectionVector4 selectionVector4 = priorityQueue.getSv4();
     SimpleSV4RecordBatch batch = new SimpleSV4RecordBatch(c, selectionVector4, context);
@@ -353,13 +350,11 @@ public class TopNBatch extends AbstractRecordBatch<TopN> {
     } else {
       for (VectorWrapper<?> i : batch) {
 
-        @SuppressWarnings("resource")
         ValueVector v = TypeHelper.getNewVector(i.getField(), oContext.getAllocator());
         newContainer.add(v);
       }
       copier.setup(batch, newContainer);
     }
-    @SuppressWarnings("resource")
     SortRecordBatchBuilder builder = new SortRecordBatchBuilder(oContext.getAllocator());
     try {
       // Purge all the existing batches to a new batch which only holds the selected records
@@ -455,11 +450,9 @@ public class TopNBatch extends AbstractRecordBatch<TopN> {
     final Stopwatch watch = Stopwatch.createStarted();
     final VectorContainer c = priorityQueue.getHyperBatch();
     final VectorContainer newContainer = new VectorContainer(oContext);
-    @SuppressWarnings("resource")
     final SelectionVector4 selectionVector4 = priorityQueue.getSv4();
     final SimpleSV4RecordBatch batch = new SimpleSV4RecordBatch(c, selectionVector4, context);
     copier = GenericCopierFactory.createAndSetupCopier(batch, newContainer, null);
-    @SuppressWarnings("resource")
     SortRecordBatchBuilder builder = new SortRecordBatchBuilder(oContext.getAllocator());
     try {
       // Purge all the existing batches to a new batch which only holds the selected records
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WriterRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WriterRecordBatch.java
index 7c13f72..4b63946 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WriterRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WriterRecordBatch.java
@@ -136,13 +136,11 @@ public class WriterRecordBatch extends AbstractRecordBatch<Writer> {
   }
 
   private void addOutputContainerData() {
-    @SuppressWarnings("resource")
     final VarCharVector fragmentIdVector = (VarCharVector) container.getValueAccessorById(
         VarCharVector.class,
         container.getValueVectorId(SchemaPath.getSimplePath("Fragment")).getFieldIds())
       .getValueVector();
     AllocationHelper.allocate(fragmentIdVector, 1, 50);
-    @SuppressWarnings("resource")
     final BigIntVector summaryVector = (BigIntVector) container.getValueAccessorById(BigIntVector.class,
             container.getValueVectorId(SchemaPath.getSimplePath("Number of records written")).getFieldIds())
           .getValueVector();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggBatch.java
index 9f51204..0883589 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggBatch.java
@@ -110,6 +110,7 @@ public class HashAggBatch extends AbstractRecordBatch<HashAggregate> {
   }
 
   private class HashAggMemoryManager extends RecordBatchMemoryManager {
+    @SuppressWarnings("unused")
     private int valuesRowWidth = 0;
 
     HashAggMemoryManager(int outputBatchSize) {
@@ -221,6 +222,7 @@ public class HashAggBatch extends AbstractRecordBatch<HashAggregate> {
     return aggregator.getOutputCount();
   }
 
+  @SuppressWarnings("incomplete-switch")
   @Override
   public void buildSchema() throws SchemaChangeException {
     IterOutcome outcome = next(incoming);
@@ -369,7 +371,6 @@ public class HashAggBatch extends AbstractRecordBatch<HashAggregate> {
       }
 
       final MaterializedField outputField = MaterializedField.create(ne.getRef().getAsNamePart().getName(), expr.getMajorType());
-      @SuppressWarnings("resource")
       ValueVector vv = TypeHelper.getNewVector(outputField, oContext.getAllocator());
 
       // add this group-by vector to the output container
@@ -395,7 +396,7 @@ public class HashAggBatch extends AbstractRecordBatch<HashAggregate> {
       }
 
       final MaterializedField outputField = MaterializedField.create(ne.getRef().getAsNamePart().getName(), expr.getMajorType());
-      @SuppressWarnings("resource") ValueVector vv = TypeHelper.getNewVector(outputField, oContext.getAllocator());
+      ValueVector vv = TypeHelper.getNewVector(outputField, oContext.getAllocator());
       aggrOutFieldIds[i] = container.add(vv);
 
       aggrExprs[i] = new ValueVectorWriteExpression(aggrOutFieldIds[i], expr, true);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java
index 2f50dd6..6cec50e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java
@@ -214,7 +214,6 @@ public abstract class HashAggTemplate implements HashAggregator {
       return (maxOccupiedIdx + 1);
     }
 
-    @SuppressWarnings("resource")
     public BatchHolder(int batchRowCount) {
 
       aggrValuesContainer = new VectorContainer();
@@ -386,6 +385,7 @@ public abstract class HashAggTemplate implements HashAggregator {
    *  Delayed setup are the parts from setup() that can only be set after actual data arrives in incoming
    *  This data is used to compute the number of partitions.
    */
+  @SuppressWarnings("unchecked")
   private void delayedSetup() {
 
     final boolean fallbackEnabled = context.getOptions().getOption(ExecConstants.HASHAGG_FALLBACK_ENABLED_KEY).bool_val;
@@ -786,7 +786,6 @@ public abstract class HashAggTemplate implements HashAggregator {
     long allocatedBefore = allocator.getAllocatedMemory();
 
     while (outgoingIter.hasNext()) {
-      @SuppressWarnings("resource")
       ValueVector vv = outgoingIter.next().getValueVector();
 
       AllocationHelper.allocatePrecomputedChildCount(vv, records, maxColumnWidth, 0);
@@ -1060,6 +1059,7 @@ public abstract class HashAggTemplate implements HashAggregator {
    *
    * @return iteration outcome (e.g., OK, NONE ...)
    */
+  @SuppressWarnings("unused")
   @Override
   public AggIterOutcome outputCurrentBatch() {
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java
index 7570016..471904f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java
@@ -391,7 +391,6 @@ public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
    * as we want the output to be NULL. For the required vectors (only for count()) we set the value to be zero since
    * we don't zero out our buffers initially while allocating them.
    */
-  @SuppressWarnings("resource")
   private void constructSpecialBatch() {
     int exprIndex = 0;
     for (final VectorWrapper<?> vw: container) {
@@ -469,7 +468,6 @@ public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
       keyExprs[i] = expr;
       final MaterializedField outputField = MaterializedField.create(ne.getRef().getLastSegment().getNameSegment().getPath(),
                                                                       expr.getMajorType());
-      @SuppressWarnings("resource")
       final ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
       keyOutputIds[i] = container.add(vector);
     }
@@ -502,7 +500,6 @@ public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
       } else {
         final MaterializedField outputField = MaterializedField.create(ne.getRef().getLastSegment().getNameSegment().getPath(),
             expr.getMajorType());
-        @SuppressWarnings("resource")
         ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
         TypedFieldId id = container.add(vector);
         valueExprs[i] = new ValueVectorWriteExpression(id, expr, true);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/ChainedHashTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/ChainedHashTable.java
index e7abd98..a0a6335 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/ChainedHashTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/ChainedHashTable.java
@@ -213,7 +213,6 @@ public class ChainedHashTable {
       LogicalExpression expr = keyExprsBuild[i];
       final MaterializedField outputField = MaterializedField.create(ne.getRef().getLastSegment().getNameSegment().getPath(),
                                                                       expr.getMajorType());
-      @SuppressWarnings("resource")
       ValueVector vv = TypeHelper.getNewVector(outputField, allocator);
       htKeyFieldIds[i] = htContainerOrig.add(vv);
       i++;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTableTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTableTemplate.java
index 1d83239..d0789e8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTableTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTableTemplate.java
@@ -187,6 +187,7 @@ public abstract class HashTableTemplate implements HashTable {
       }
     }
 
+    @SuppressWarnings("unused")
     private void init(IntVector links, IntVector hashValues, int size) {
       for (int i = 0; i < size; i++) {
         links.getMutator().set(i, EMPTY_SLOT);
@@ -363,9 +364,7 @@ public abstract class HashTableTemplate implements HashTable {
       Iterator<VectorWrapper<?>> outgoingIter = outContainer.iterator();
 
       for (VectorWrapper<?> sourceWrapper : htContainer) {
-        @SuppressWarnings("resource")
         ValueVector sourceVV = sourceWrapper.getValueVector();
-        @SuppressWarnings("resource")
         ValueVector targetVV = outgoingIter.next().getValueVector();
         TransferPair tp = sourceVV.makeTransferPair(targetVV);
         // The normal case: The whole column key(s) are transfered as is
@@ -376,7 +375,6 @@ public abstract class HashTableTemplate implements HashTable {
 
     private void setValueCount() {
       for (VectorWrapper<?> vw : htContainer) {
-        @SuppressWarnings("resource")
         ValueVector vv = vw.getValueVector();
         vv.getMutator().setValueCount(maxOccupiedIdx + 1);
       }
@@ -404,6 +402,7 @@ public abstract class HashTableTemplate implements HashTable {
 
     // Only used for internal debugging. Get the value vector at a particular index from the htContainer.
     // By default this assumes the VV is a BigIntVector.
+    @SuppressWarnings("unused")
     private ValueVector getValueVector(int index) {
       Object tmp = (htContainer).getValueAccessorById(BigIntVector.class, index).getValueVector();
       if (tmp != null) {
@@ -881,6 +880,7 @@ public abstract class HashTableTemplate implements HashTable {
    * Reinit the hash table to its original size, and clear up all its prior batch holder
    *
    */
+  @Override
   public void reset() {
     this.clear(false); // Clear all current batch holders and hash table (i.e. free their memory)
 
@@ -893,6 +893,7 @@ public abstract class HashTableTemplate implements HashTable {
     startIndices = allocMetadataVector(originalTableSize, EMPTY_SLOT);
   }
 
+  @Override
   public void updateIncoming(VectorContainer newIncoming, RecordBatch newIncomingProbe) {
     incomingBuild = newIncoming;
     incomingProbe = newIncomingProbe;
@@ -920,6 +921,7 @@ public abstract class HashTableTemplate implements HashTable {
     return vector;
   }
 
+  @Override
   public Pair<VectorContainer, Integer> nextBatch() {
     if (batchHolders == null || batchHolders.size() == 0) {
       return null;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/flatten/FlattenRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/flatten/FlattenRecordBatch.java
index 86ddcd1..56f8f2b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/flatten/FlattenRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/flatten/FlattenRecordBatch.java
@@ -204,7 +204,6 @@ public class FlattenRecordBatch extends AbstractSingleRecordBatch<FlattenPOP> {
     return this.container;
   }
 
-  @SuppressWarnings("resource")
   private void setFlattenVector() {
     final TypedFieldId typedFieldId = incoming.getValueVectorId(popConfig.getColumn());
     final MaterializedField field = incoming.getSchema().getColumn(typedFieldId.getFieldIds()[0]);
@@ -359,7 +358,6 @@ public class FlattenRecordBatch extends AbstractSingleRecordBatch<FlattenPOP> {
    * the end of one of the other vectors while we are copying the data of the other vectors alongside each new flattened
    * value coming out of the repeated field.)
    */
-  @SuppressWarnings("resource")
   private TransferPair getFlattenFieldTransferPair(FieldReference reference) {
     final TypedFieldId fieldId = incoming.getValueVectorId(popConfig.getColumn());
     final Class<?> vectorClass = incoming.getSchema().getColumn(fieldId.getFieldIds()[0]).getValueClass();
@@ -456,7 +454,6 @@ public class FlattenRecordBatch extends AbstractSingleRecordBatch<FlattenPOP> {
         final MaterializedField outputField;
         if (expr instanceof ValueVectorReadExpression) {
           final TypedFieldId id = ValueVectorReadExpression.class.cast(expr).getFieldId();
-          @SuppressWarnings("resource")
           final ValueVector incomingVector = incoming.getValueAccessorById(id.getIntermediateClass(), id.getFieldIds()).getValueVector();
           // outputField is taken from the incoming schema to avoid the loss of nested fields
           // when the first batch will be empty.
@@ -468,7 +465,6 @@ public class FlattenRecordBatch extends AbstractSingleRecordBatch<FlattenPOP> {
         } else {
           outputField = MaterializedField.create(outputName, expr.getMajorType());
         }
-        @SuppressWarnings("resource")
         final ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
         allocationVectors.add(vector);
         TypedFieldId fid = container.add(vector);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
index 97f4b96..8f16b51 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
@@ -142,7 +142,6 @@ public class MergingRecordBatch extends AbstractRecordBatch<MergingReceiverPOP>
     context.getBuffers().getCollector(config.getOppositeMajorFragmentId()).setAllocator(oContext.getAllocator());
   }
 
-  @SuppressWarnings("resource")
   private RawFragmentBatch getNext(final int providerIndex) throws IOException {
     stats.startWait();
     final RawFragmentBatchProvider provider = fragProviders[providerIndex];
@@ -568,7 +567,6 @@ public class MergingRecordBatch extends AbstractRecordBatch<MergingReceiverPOP>
         }
         tempBatchHolder[i] = batch;
         for (final SerializedField field : batch.getHeader().getDef().getFieldList()) {
-          @SuppressWarnings("resource")
           final ValueVector v = outgoingContainer.addOrGet(MaterializedField.create(field));
           v.allocateNew();
         }
@@ -692,7 +690,6 @@ public class MergingRecordBatch extends AbstractRecordBatch<MergingReceiverPOP>
 
   private void allocateOutgoing() {
     for (final VectorWrapper<?> w : outgoingContainer) {
-      @SuppressWarnings("resource")
       final ValueVector v = w.getValueVector();
       if (v instanceof FixedWidthVector) {
         AllocationHelper.allocate(v, OUTGOING_BATCH_SIZE, 1);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java
index 63a0121..4881d4b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java
@@ -162,7 +162,6 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
   }
 
 
-  @SuppressWarnings("resource")
   private boolean saveSamples() throws SchemaChangeException, ClassTransformationException, IOException {
     recordsSampled = 0;
     IterOutcome upstream;
@@ -340,7 +339,6 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
 
     // Get all samples from distributed map
 
-    @SuppressWarnings("resource")
     SortRecordBatchBuilder containerBuilder = new SortRecordBatchBuilder(context.getAllocator());
     final VectorContainer allSamplesContainer = new VectorContainer();
     final VectorContainer candidatePartitionTable = new VectorContainer();
@@ -359,7 +357,6 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
       }
 
       // sort the data incoming samples.
-      @SuppressWarnings("resource")
       SelectionVector4 newSv4 = containerBuilder.getSv4();
       Sorter sorter = SortBatch.createNewSorter(context, orderDefs, allSamplesContainer);
       sorter.setup(context, newSv4, allSamplesContainer);
@@ -388,7 +385,6 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
         }
       }
       candidatePartitionTable.setRecordCount(copier.getOutputRecords());
-      @SuppressWarnings("resource")
       WritableBatch batch = WritableBatch.getBatchNoHVWrap(candidatePartitionTable.getRecordCount(), candidatePartitionTable, false);
       wrap = new CachedVectorContainer(batch, context.getAllocator());
       tableMap.putIfAbsent(mapKey + "final", wrap, 1, TimeUnit.MINUTES);
@@ -439,7 +435,6 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
             "Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
       }
 
-      @SuppressWarnings("resource")
       ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
       localAllocationVectors.add(vector);
       TypedFieldId fid = outgoing.add(vector);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionSenderCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionSenderCreator.java
index 8e6ea0b..14ae6b0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionSenderCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionSenderCreator.java
@@ -59,7 +59,6 @@ import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
  */
 public class OrderedPartitionSenderCreator implements RootCreator<OrderedPartitionSender> {
 
-  @SuppressWarnings("resource")
   @Override
   public RootExec getRoot(ExecutorFragmentContext context, OrderedPartitionSender config,
       List<RecordBatch> children) throws ExecutionSetupException {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionerTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionerTemplate.java
index 687ff81..edbbfe2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionerTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionerTemplate.java
@@ -389,7 +389,6 @@ public abstract class PartitionerTemplate implements Partitioner {
     public void initializeBatch() {
       for (VectorWrapper<?> v : incoming) {
         // create new vector
-        @SuppressWarnings("resource")
         ValueVector outgoingVector = TypeHelper.getNewVector(v.getField(), allocator);
         outgoingVector.setInitialCapacity(outgoingRecordBatchSize);
         vectorContainer.add(outgoingVector);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/ColumnsArrayManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/ColumnsArrayManager.java
index d315466..470929c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/ColumnsArrayManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/ColumnsArrayManager.java
@@ -21,7 +21,7 @@ import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.exec.physical.impl.scan.project.ColumnProjection;
 import org.apache.drill.exec.physical.impl.scan.project.ResolvedTuple;
 import org.apache.drill.exec.physical.impl.scan.project.ScanLevelProjection.ScanProjectionParser;
-import org.apache.drill.exec.physical.impl.scan.project.SchemaLevelProjection.SchemaProjectionResolver;
+import org.apache.drill.exec.physical.impl.scan.project.ReaderLevelProjection.ReaderProjectionResolver;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
 
@@ -74,7 +74,7 @@ import org.apache.drill.exec.record.metadata.TupleMetadata;
  * items to the array column via the usual vector writer mechanism.
  */
 
-public class ColumnsArrayManager implements SchemaProjectionResolver {
+public class ColumnsArrayManager implements ReaderProjectionResolver {
 
   public static final String COLUMNS_COL = "columns";
 
@@ -88,7 +88,7 @@ public class ColumnsArrayManager implements SchemaProjectionResolver {
 
   public ScanProjectionParser projectionParser() { return parser; }
 
-  public SchemaProjectionResolver resolver() { return this; }
+  public ReaderProjectionResolver resolver() { return this; }
 
   @Override
   public void startResolution() {
@@ -97,7 +97,7 @@ public class ColumnsArrayManager implements SchemaProjectionResolver {
   @Override
   public boolean resolveColumn(ColumnProjection col, ResolvedTuple outputTuple,
       TupleMetadata tableSchema) {
-    if (col.nodeType() != UnresolvedColumnsArrayColumn.ID) {
+    if (! (col instanceof UnresolvedColumnsArrayColumn)) {
       return false;
     }
 
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 b9b3e78..383de52 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
@@ -21,7 +21,7 @@ import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.physical.impl.scan.project.ColumnProjection;
 import org.apache.drill.exec.physical.impl.scan.project.ScanLevelProjection;
 import org.apache.drill.exec.physical.impl.scan.project.ScanLevelProjection.ScanProjectionParser;
-import org.apache.drill.exec.physical.impl.scan.project.UnresolvedColumn;
+import org.apache.drill.exec.physical.impl.scan.project.AbstractUnresolvedColumn.UnresolvedColumn;
 import org.apache.drill.exec.physical.rowSet.project.RequestedColumnImpl;
 import org.apache.drill.exec.physical.rowSet.project.RequestedTuple.RequestedColumn;
 import org.apache.drill.exec.store.easy.text.compliant.v3.TextReader;
@@ -145,7 +145,7 @@ public class ColumnsArrayParser implements ScanProjectionParser {
 
   @Override
   public void validateColumn(ColumnProjection col) {
-    if (col.nodeType() == UnresolvedColumn.UNRESOLVED) {
+    if (col instanceof UnresolvedColumn) {
       if (columnsArrayCol != null) {
         throw UserException
           .validationError()
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/ColumnsScanFramework.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/ColumnsScanFramework.java
index 17265c8..7efe221 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/ColumnsScanFramework.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/ColumnsScanFramework.java
@@ -17,18 +17,8 @@
  */
 package org.apache.drill.exec.physical.impl.scan.columns;
 
-import java.util.List;
-
-import org.apache.drill.common.exceptions.ExecutionSetupException;
-import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.exec.physical.impl.scan.file.BaseFileScanFramework;
-import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiatorImpl;
-import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
-import org.apache.drill.exec.physical.impl.scan.framework.ShimBatchReader;
-import org.apache.drill.exec.store.dfs.DrillFileSystem;
-import org.apache.drill.exec.store.dfs.easy.FileWork;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapred.FileSplit;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.SchemaNegotiatorImpl;
 
 /**
  * Scan framework for a file that supports the special "columns" column.
@@ -43,12 +33,19 @@ import org.apache.hadoop.mapred.FileSplit;
  * identifier, the use of the columns identifier when it is not allowed, etc.
  */
 
-public class ColumnsScanFramework extends BaseFileScanFramework<ColumnsSchemaNegotiator> {
+public class ColumnsScanFramework extends FileScanFramework {
 
-  public interface FileReaderCreator {
-    ManagedReader<ColumnsSchemaNegotiator> makeBatchReader(
-        DrillFileSystem dfs,
-        FileSplit split) throws ExecutionSetupException;
+  public static class ColumnsScanBuilder extends FileScanBuilder {
+    protected boolean requireColumnsArray;
+
+    public void requireColumnsArray(boolean flag) {
+      requireColumnsArray = flag;
+    }
+
+    @Override
+    public FileScanFramework buildFileFramework() {
+      return new ColumnsScanFramework(this);
+    }
   }
 
   /**
@@ -58,13 +55,12 @@ public class ColumnsScanFramework extends BaseFileScanFramework<ColumnsSchemaNeg
   public static class ColumnsSchemaNegotiatorImpl extends FileSchemaNegotiatorImpl
           implements ColumnsSchemaNegotiator {
 
-    public ColumnsSchemaNegotiatorImpl(ColumnsScanFramework framework,
-        ShimBatchReader<ColumnsSchemaNegotiator> shim) {
-      super(framework, shim);
+    public ColumnsSchemaNegotiatorImpl(ColumnsScanFramework framework) {
+      super(framework);
     }
 
     private ColumnsScanFramework framework() {
-      return (ColumnsScanFramework) basicFramework;
+      return (ColumnsScanFramework) framework;
     }
 
     @Override
@@ -78,46 +74,28 @@ public class ColumnsScanFramework extends BaseFileScanFramework<ColumnsSchemaNeg
     }
   }
 
-  /**
-   * Creates (or iterates over) the readers for this scan.
-   */
-  private final FileReaderCreator readerCreator;
-  private boolean requireColumnsArray;
   protected ColumnsArrayManager columnsArrayManager;
 
-  public ColumnsScanFramework(List<SchemaPath> projection,
-      List<? extends FileWork> files,
-      Configuration fsConf,
-      FileReaderCreator readerCreator) {
-    super(projection, files, fsConf);
-    this.readerCreator = readerCreator;
-  }
-
-  public void requireColumnsArray(boolean flag) {
-    requireColumnsArray = flag;
+  public ColumnsScanFramework(ColumnsScanBuilder builder) {
+    super(builder);
   }
 
   @Override
   protected void configure() {
     super.configure();
-    columnsArrayManager = new ColumnsArrayManager(requireColumnsArray);
-    scanOrchestrator.addParser(columnsArrayManager.projectionParser());
-    scanOrchestrator.addResolver(columnsArrayManager.resolver());
+    columnsArrayManager = new ColumnsArrayManager(
+       ((ColumnsScanBuilder) builder).requireColumnsArray);
+    builder.addParser(columnsArrayManager.projectionParser());
+    builder.addResolver(columnsArrayManager.resolver());
 
     // This framework is (at present) used only for the text readers
     // which use required Varchar columns to represent null columns.
 
-    scanOrchestrator.allowRequiredNullColumns(true);
-  }
-
-  @Override
-  protected ManagedReader<ColumnsSchemaNegotiator> newReader(FileSplit split) throws ExecutionSetupException {
-    return readerCreator.makeBatchReader(dfs, split);
+    builder.allowRequiredNullColumns(true);
   }
 
   @Override
-  public boolean openReader(ShimBatchReader<ColumnsSchemaNegotiator> shim, ManagedReader<ColumnsSchemaNegotiator> reader) {
-    return reader.open(
-        new ColumnsSchemaNegotiatorImpl(this, shim));
+  protected SchemaNegotiatorImpl newNegotiator() {
+    return new ColumnsSchemaNegotiatorImpl(this);
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/ColumnsSchemaNegotiator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/ColumnsSchemaNegotiator.java
index 3bffd8a..8d48927 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/ColumnsSchemaNegotiator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/ColumnsSchemaNegotiator.java
@@ -17,15 +17,14 @@
  */
 package org.apache.drill.exec.physical.impl.scan.columns;
 
-import org.apache.drill.exec.physical.impl.scan.file.BaseFileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
 
 /**
  * Schema negotiator that supports the file scan options plus access
  * to the specific selected columns indexes.
  */
-public interface ColumnsSchemaNegotiator extends BaseFileScanFramework.FileSchemaNegotiator {
+public interface ColumnsSchemaNegotiator extends FileSchemaNegotiator {
 
   boolean columnsArrayProjected();
   boolean[] projectedIndexes();
-
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/ResolvedColumnsArrayColumn.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/ResolvedColumnsArrayColumn.java
index 38f32f3..1e9b627 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/ResolvedColumnsArrayColumn.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/ResolvedColumnsArrayColumn.java
@@ -24,8 +24,6 @@ import org.apache.drill.exec.record.MaterializedField;
 
 public class ResolvedColumnsArrayColumn extends ResolvedTableColumn {
 
-  public static final int ID = 21;
-
   private final RequestedColumn inCol;
 
   public ResolvedColumnsArrayColumn(UnresolvedColumnsArrayColumn unresolved,
@@ -35,8 +33,5 @@ public class ResolvedColumnsArrayColumn extends ResolvedTableColumn {
     inCol = unresolved.element();
   }
 
-  @Override
-  public int nodeType() { return ID; }
-
   public boolean[] selectedIndexes() { return inCol.indexes(); }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/UnresolvedColumnsArrayColumn.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/UnresolvedColumnsArrayColumn.java
index 50eb222..570a2d1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/UnresolvedColumnsArrayColumn.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/UnresolvedColumnsArrayColumn.java
@@ -17,15 +17,13 @@
  */
 package org.apache.drill.exec.physical.impl.scan.columns;
 
-import org.apache.drill.exec.physical.impl.scan.project.UnresolvedColumn;
+import org.apache.drill.exec.physical.impl.scan.project.AbstractUnresolvedColumn;
 import org.apache.drill.exec.physical.rowSet.project.RequestedTuple.RequestedColumn;
 
-public class UnresolvedColumnsArrayColumn extends UnresolvedColumn {
-
-  public static final int ID = 20;
+public class UnresolvedColumnsArrayColumn extends AbstractUnresolvedColumn {
 
   public UnresolvedColumnsArrayColumn(RequestedColumn inCol) {
-    super(inCol, ID);
+    super(inCol);
   }
 
   public boolean[] selectedIndexes() { return inCol.indexes(); }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/BaseFileScanFramework.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/BaseFileScanFramework.java
deleted file mode 100644
index 1166a52..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/BaseFileScanFramework.java
+++ /dev/null
@@ -1,176 +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.file;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.drill.common.exceptions.ExecutionSetupException;
-import org.apache.drill.common.exceptions.UserException;
-import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.exec.physical.impl.scan.RowBatchReader;
-import org.apache.drill.exec.physical.impl.scan.framework.AbstractScanFramework;
-import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
-import org.apache.drill.exec.physical.impl.scan.framework.SchemaNegotiator;
-import org.apache.drill.exec.physical.impl.scan.framework.ShimBatchReader;
-import org.apache.drill.exec.store.dfs.DrillFileSystem;
-import org.apache.drill.exec.store.dfs.easy.FileWork;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.FileSplit;
-
-/**
- * Scan framework for a file that implements metadata columns (AKA "implicit"
- * columns and partition columns.)
- * <p>
- * Framework iterators over file descriptions, creating readers at the
- * moment they are needed. This allows simpler logic because, at the point of
- * reader creation, we have a file system, context and so on.
- */
-
-public abstract class BaseFileScanFramework<T extends BaseFileScanFramework.FileSchemaNegotiator>
-    extends AbstractScanFramework<T> {
-
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BaseFileScanFramework.class);
-
-  /**
-   * The file schema negotiator adds no behavior at present, but is
-   * created as a placeholder anticipating the need for file-specific
-   * behavior later. Readers are expected to use an instance of this
-   * class so that their code need not change later if/when we add new
-   * methods. For example, perhaps we want to specify an assumed block
-   * size for S3 files, or want to specify behavior if the file no longer
-   * exists. Those are out of scope of this first round of changes which
-   * focus on schema.
-   */
-
-  public interface FileSchemaNegotiator extends SchemaNegotiator {
-  }
-
-  private final List<? extends FileWork> files;
-  private final Configuration fsConfig;
-  private List<FileSplit> spilts = new ArrayList<>();
-  private Iterator<FileSplit> splitIter;
-  private Path scanRootDir;
-  private int partitionDepth;
-  protected DrillFileSystem dfs;
-  private FileMetadataManager metadataManager;
-
-  public BaseFileScanFramework(List<SchemaPath> projection,
-      List<? extends FileWork> files,
-      Configuration fsConf) {
-    super(projection);
-    this.files = files;
-    this.fsConfig = fsConf;
-  }
-
-  /**
-   * Specify the selection root for a directory scan, if any.
-   * Used to populate partition columns. Also, specify the maximum
-   * partition depth.
-   *
-   * @param rootPath Hadoop file path for the directory
-   * @param partitionDepth maximum partition depth across all files
-   * within this logical scan operator (files in this scan may be
-   * shallower)
-   */
-
-  public void setSelectionRoot(Path rootPath, int partitionDepth) {
-    this.scanRootDir = rootPath;
-    this.partitionDepth = partitionDepth;
-  }
-
-  @Override
-  protected void configure() {
-    super.configure();
-
-    // Create the Drill file system.
-
-    try {
-      dfs = context.newFileSystem(fsConfig);
-    } catch (IOException e) {
-      throw UserException.dataReadError(e)
-        .addContext("Failed to create FileSystem")
-        .build(logger);
-    }
-
-    // Prepare the list of files. We need the list of paths up
-    // front to compute the maximum partition. Then, we need to
-    // iterate over the splits to create readers on demand.
-
-    List<Path> paths = new ArrayList<>();
-    for (FileWork work : files) {
-      Path path = dfs.makeQualified(work.getPath());
-      paths.add(path);
-      FileSplit split = new FileSplit(path, work.getStart(), work.getLength(), new String[]{""});
-      spilts.add(split);
-    }
-    splitIter = spilts.iterator();
-
-    // Create the metadata manager to handle file metadata columns
-    // (so-called implicit columns and partition columns.)
-
-    metadataManager = new FileMetadataManager(
-        context.getFragmentContext().getOptions(),
-        true, // Expand partition columns with wildcard
-        false, // Put partition columns after table columns
-        scanRootDir,
-        partitionDepth,
-        paths);
-    scanOrchestrator.withMetadata(metadataManager);
-  }
-
-  @Override
-  public RowBatchReader nextReader() {
-
-    // Create a reader on demand for the next split.
-
-    if (! splitIter.hasNext()) {
-      return null;
-    }
-    FileSplit split = splitIter.next();
-
-    // Alert the framework that a new file is starting.
-
-    startFile(split);
-    try {
-
-      // Create a per-framework reader wrapped in a standard
-      // "shim" reader. Allows app-specific readers to be very focused;
-      // the shim handles standard boilerplate.
-
-      return new ShimBatchReader<T>(this, newReader(split));
-    } catch (ExecutionSetupException e) {
-      throw UserException.executionError(e)
-        .addContext("File", split.getPath().toString())
-        .build(logger);
-    }
-  }
-
-  protected abstract ManagedReader<T> newReader(FileSplit split) throws ExecutionSetupException;
-
-  protected void startFile(FileSplit split) {
-
-    // Tell the metadata manager about the current file so it can
-    // populate the metadata columns, if requested.
-
-    metadataManager.startFile(split.getPath());
-  }
-}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileMetadataColumn.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileMetadataColumn.java
index ad2e271..1c55d86 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileMetadataColumn.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileMetadataColumn.java
@@ -21,14 +21,12 @@ import org.apache.drill.exec.physical.impl.scan.project.VectorSource;
 
 /**
  * Represents projection column which resolved to a file metadata
- * (AKA "implicit") column sch as "filename", "fqn", etc. These
+ * (AKA "implicit") column such as "filename", "fqn", etc. These
  * columns are "synthetic" added by the scan framework itself
  * rather than "organic" coming from the scanned table.
  */
 public class FileMetadataColumn extends MetadataColumn {
 
-  public static final int ID = 15;
-
   private final FileMetadataColumnDefn defn;
 
   /**
@@ -63,9 +61,6 @@ public class FileMetadataColumn extends MetadataColumn {
     this.defn = defn;
   }
 
-  @Override
-  public int nodeType() { return ID; }
-
   public FileMetadataColumnDefn defn() { return defn; }
 
   @Override
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 a4ace55..3289c2c 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
@@ -125,10 +125,10 @@ public class FileMetadataColumnsParser implements ScanProjectionParser {
   }
 
   private void buildWildcard() {
-    if (!metadataManager.useLegacyWildcardExpansion) {
+    if (!metadataManager.options().useLegacyWildcardExpansion) {
       return;
     }
-    if (metadataManager.useLegacyExpansionLocation) {
+    if (metadataManager.options().useLegacyExpansionLocation) {
 
       // Star column: this is a SELECT * query.
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileMetadataManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileMetadataManager.java
index 201d308..619565f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileMetadataManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileMetadataManager.java
@@ -29,7 +29,7 @@ import org.apache.drill.exec.physical.impl.scan.project.ConstantColumnLoader;
 import org.apache.drill.exec.physical.impl.scan.project.MetadataManager;
 import org.apache.drill.exec.physical.impl.scan.project.ResolvedTuple;
 import org.apache.drill.exec.physical.impl.scan.project.ScanLevelProjection.ScanProjectionParser;
-import org.apache.drill.exec.physical.impl.scan.project.SchemaLevelProjection.SchemaProjectionResolver;
+import org.apache.drill.exec.physical.impl.scan.project.ReaderLevelProjection.ReaderProjectionResolver;
 import org.apache.drill.exec.physical.impl.scan.project.VectorSource;
 import org.apache.drill.exec.physical.rowSet.ResultVectorCache;
 import org.apache.drill.exec.record.VectorContainer;
@@ -61,7 +61,7 @@ import org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTes
  * This is the successor to {@link ColumnExplorer}.
  */
 
-public class FileMetadataManager implements MetadataManager, SchemaProjectionResolver, VectorSource {
+public class FileMetadataManager implements MetadataManager, ReaderProjectionResolver, VectorSource {
 
   /**
    * Automatically compute partition depth from files. Use only
@@ -70,41 +70,78 @@ public class FileMetadataManager implements MetadataManager, SchemaProjectionRes
 
   public static final int AUTO_PARTITION_DEPTH = -1;
 
+  public static class FileMetadataOptions {
+
+    private Path rootDir;
+    private int partitionCount = AUTO_PARTITION_DEPTH;
+    private List<Path> files;
+    protected boolean useLegacyWildcardExpansion = true;
+    protected boolean useLegacyExpansionLocation;
+
+    /**
+      * Specify the selection root for a directory scan, if any.
+      * Used to populate partition columns. Also, specify the maximum
+      * partition depth.
+      *
+      * @param rootPath Hadoop file path for the directory
+      * @param partitionDepth maximum partition depth across all files
+      * within this logical scan operator (files in this scan may be
+      * shallower)
+      */
+
+     public void setSelectionRoot(Path rootPath) {
+       this.rootDir = rootPath;
+     }
+
+     public void setPartitionDepth(int partitionDepth) {
+       this.partitionCount = partitionDepth;
+     }
+
+     public void setFiles(List<Path> files) {
+      this.files = files;
+    }
+
+     /**
+      * Indicates whether to expand partition columns when the query contains a wildcard.
+      * Supports queries such as the following:<code><pre>
+      * select * from dfs.`partitioned-dir`</pre></code>
+      * In which the output columns will be (columns, dir0) if the partitioned directory
+      * has one level of nesting.
+      *
+      * See {@link TestImplicitFileColumns#testImplicitColumns}
+      */
+     public void useLegacyWildcardExpansion(boolean flag) {
+       useLegacyWildcardExpansion = flag;
+     }
+
+     /**
+      * In legacy mode, above, Drill expands partition columns whenever the
+      * wildcard appears. Drill 1.1 - 1.11 put expanded partition columns after
+      * data columns. This is actually a better position as it minimizes changes
+      * the row layout for files at different depths. Drill 1.12 moved them before
+      * data columns: at the location of the wildcard.
+      * <p>
+      * This flag, when set, uses the Drill 1.12 position. Later enhancements
+      * can unset this flag to go back to the future: use the preferred location
+      * after other columns.
+      */
+     public void useLegacyExpansionLocation(boolean flag) {
+       useLegacyExpansionLocation = flag;
+     }
+  }
+
   // Input
 
-  private final Path scanRootDir;
-  private final int partitionCount;
+  private final FileMetadataOptions options;
   private FileMetadata currentFile;
 
   // Config
 
+  private final Path scanRootDir;
+  private final int partitionCount;
   protected final String partitionDesignator;
   protected final List<FileMetadataColumnDefn> implicitColDefns = new ArrayList<>();
   protected final Map<String, FileMetadataColumnDefn> fileMetadataColIndex = CaseInsensitiveMap.newHashMap();
-
-  /**
-   * Indicates whether to expand partition columns when the query contains a wildcard.
-   * Supports queries such as the following:<code><pre>
-   * select * from dfs.`partitioned-dir`</pre></code>
-   * In which the output columns will be (columns, dir0) if the partitioned directory
-   * has one level of nesting.
-   *
-   * See {@link TestImplicitFileColumns#testImplicitColumns}
-   */
-  protected final boolean useLegacyWildcardExpansion;
-
-  /**
-   * In legacy mode, above, Drill expands partition columns whenever the
-   * wildcard appears. Drill 1.1 - 1.11 put expanded partition columns after
-   * data columns. This is actually a better position as it minimizes changes
-   * the row layout for files at different depths. Drill 1.12 moved them before
-   * data columns: at the location of the wildcard.
-   * <p>
-   * This flag, when set, uses the Drill 1.12 position. Later enhancements
-   * can unset this flag to go back to the future: use the preferred location
-   * after other columns.
-   */
-  protected final boolean useLegacyExpansionLocation;
   private final FileMetadataColumnsParser parser;
 
   // Internal state
@@ -135,13 +172,8 @@ public class FileMetadataManager implements MetadataManager, SchemaProjectionRes
    */
 
   public FileMetadataManager(OptionSet optionManager,
-      boolean useLegacyWildcardExpansion,
-      boolean useLegacyExpansionLocation,
-      Path rootDir,
-      int partitionCount,
-      List<Path> files) {
-    this.useLegacyWildcardExpansion = useLegacyWildcardExpansion;
-    this.useLegacyExpansionLocation = useLegacyExpansionLocation;
+      FileMetadataOptions config) {
+    this.options = config;
 
     partitionDesignator = optionManager.getString(ExecConstants.FILESYSTEM_PARTITION_COLUMN_LABEL);
     for (ImplicitFileColumns e : ImplicitFileColumns.values()) {
@@ -156,32 +188,34 @@ public class FileMetadataManager implements MetadataManager, SchemaProjectionRes
 
     // The files and root dir are optional.
 
-    if (rootDir == null || files == null) {
+    if (config.rootDir == null || config.files == null) {
       scanRootDir = null;
-      this.partitionCount = 0;
+      partitionCount = 0;
 
     // Special case in which the file is the same as the
     // root directory (occurs for a query with only one file.)
 
-    } else if (files.size() == 1 && rootDir.equals(files.get(0))) {
+    } else if (config.files.size() == 1 && config.rootDir.equals(config.files.get(0))) {
       scanRootDir = null;
-      this.partitionCount = 0;
+      partitionCount = 0;
     } else {
-      scanRootDir = rootDir;
+      scanRootDir = config.rootDir;
 
       // Compute the partitions. Normally the count is passed in.
       // But, handle the case where the count is unknown. Note: use this
       // convenience only in testing since, in production, it can result
       // in different scans reporting different numbers of partitions.
 
-      if (partitionCount == -1) {
-        this.partitionCount = computeMaxPartition(files);
+      if (config.partitionCount == -1) {
+        partitionCount = computeMaxPartition(config.files);
       } else {
-        this.partitionCount = partitionCount;
+        partitionCount = options.partitionCount;
       }
     }
   }
 
+  protected FileMetadataOptions options() { return options; }
+
   private int computeMaxPartition(List<Path> files) {
     int maxLen = 0;
     for (Path filePath : files) {
@@ -227,7 +261,7 @@ public class FileMetadataManager implements MetadataManager, SchemaProjectionRes
   }
 
   @Override
-  public SchemaProjectionResolver resolver() { return this; }
+  public ReaderProjectionResolver resolver() { return this; }
 
   @Override
   public void define() {
@@ -272,17 +306,12 @@ public class FileMetadataManager implements MetadataManager, SchemaProjectionRes
   @Override
   public boolean resolveColumn(ColumnProjection col, ResolvedTuple tuple,
       TupleMetadata tableSchema) {
-    MetadataColumn outputCol = null;
-    switch (col.nodeType()) {
-    case PartitionColumn.ID:
+    MetadataColumn outputCol;
+    if (col instanceof PartitionColumn) {
       outputCol = ((PartitionColumn) col).resolve(currentFile, this, metadataColumns.size());
-      break;
-
-    case FileMetadataColumn.ID:
+    } else if (col instanceof FileMetadataColumn) {
       outputCol = ((FileMetadataColumn) col).resolve(currentFile, this, metadataColumns.size());
-      break;
-
-    default:
+    } else {
       return false;
     }
 
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 6ecf0cf..e1b3374 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
@@ -17,27 +17,31 @@
  */
 package org.apache.drill.exec.physical.impl.scan.file;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
 import java.util.List;
 
-import org.apache.drill.common.exceptions.ExecutionSetupException;
-import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.exec.physical.impl.scan.file.BaseFileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.physical.impl.scan.file.FileMetadataManager.FileMetadataOptions;
 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;
 import org.apache.drill.exec.physical.impl.scan.framework.SchemaNegotiatorImpl;
 import org.apache.drill.exec.physical.impl.scan.framework.ShimBatchReader;
 import org.apache.drill.exec.store.dfs.DrillFileSystem;
 import org.apache.drill.exec.store.dfs.easy.FileWork;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.FileSplit;
 
 /**
  * The file scan framework adds into the scan framework support for implicit
- * file metadata columns. The file scan framework brings together a number of
- * components:
+ * reading from DFS splits (a file and a block). Since this framework is
+ * file-based, it also adds support for file metadata (AKA implicit columns.
+ * The file scan framework brings together a number of components:
  * <ul>
- * <li>The projection list provided by the physical operator definition. This
- * list identifies the set of "output" columns whih this framework is obliged
- * to produce.</li>
+ * <li>The set of options defined by the base framework.</li>
  * <li>The set of files and/or blocks to read.</li>
  * <li>The file system configuration to use for working with the files
  * or blocks.</li>
@@ -46,26 +50,29 @@ import org.apache.hadoop.mapred.FileSplit;
  * <li>Options as defined by the base class.</li>
  * </ul>
  * <p>
+ * The framework iterates over file descriptions, creating readers at the
+ * moment they are needed. This allows simpler logic because, at the point of
+ * reader creation, we have a file system, context and so on.
+ * <p>
  * @See {AbstractScanFramework} for details.
  */
 
-public class FileScanFramework extends BaseFileScanFramework<FileSchemaNegotiator> {
+public class FileScanFramework extends ManagedScanFramework {
+
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FileScanFramework.class);
 
   /**
-   * Creates a batch reader on demand. Unlike earlier versions of Drill,
-   * this framework creates readers one by one, when they are needed.
-   * Doing so avoids excessive resource demands that come from creating
-   * potentially thousands of readers up front.
-   * <p>
-   * The reader itself is unique to each file type. This interface
-   * provides a common interface that this framework can use to create the
-   * file-specific reader on demand.
+   * The file schema negotiator adds no behavior at present, but is
+   * created as a placeholder anticipating the need for file-specific
+   * behavior later. Readers are expected to use an instance of this
+   * class so that their code need not change later if/when we add new
+   * methods. For example, perhaps we want to specify an assumed block
+   * size for S3 files, or want to specify behavior if the file no longer
+   * exists. Those are out of scope of this first round of changes which
+   * focus on schema.
    */
 
-  public interface FileReaderFactory {
-    ManagedReader<FileSchemaNegotiator> makeBatchReader(
-        DrillFileSystem dfs,
-        FileSplit split) throws ExecutionSetupException;
+  public interface FileSchemaNegotiator extends SchemaNegotiator {
   }
 
   /**
@@ -77,29 +84,141 @@ public class FileScanFramework extends BaseFileScanFramework<FileSchemaNegotiato
   public static class FileSchemaNegotiatorImpl extends SchemaNegotiatorImpl
       implements FileSchemaNegotiator {
 
-    public FileSchemaNegotiatorImpl(BaseFileScanFramework<?> framework, ShimBatchReader<? extends FileSchemaNegotiator> shim) {
-      super(framework, shim);
+    public FileSchemaNegotiatorImpl(ManagedScanFramework framework) {
+      super(framework);
+    }
+  }
+
+  /**
+   * 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();
+
+    public void setConfig(Configuration fsConf) {
+      this.fsConf = fsConf;
+    }
+
+    public void setFiles(List<? extends FileWork> files) {
+      this.files = files;
+    }
+
+    public FileMetadataOptions metadataOptions() { return metadataOptions; }
+
+    public FileScanFramework buildFileFramework() {
+      return new FileScanFramework(this);
+    }
+  }
+
+  public abstract static class FileReaderFactory implements ReaderFactory {
+
+    protected FileScanFramework fileFramework;
+
+    @Override
+    public void bind(ManagedScanFramework baseFramework) {
+      this.fileFramework = (FileScanFramework) baseFramework;
+    }
+
+    @Override
+    public ManagedReader<? extends SchemaNegotiator> next() {
+      FileSplit split = fileFramework.nextSplit();
+      if (split == null) {
+        return null;
+      }
+      return newReader(split);
     }
+
+    protected DrillFileSystem fileSystem() { return fileFramework.dfs; }
+
+    public abstract ManagedReader<? extends FileSchemaNegotiator> newReader(FileSplit split);
   }
 
-  private final FileReaderFactory readerCreator;
+  private FileMetadataManager metadataManager;
+  private DrillFileSystem dfs;
+  private List<FileSplit> spilts = new ArrayList<>();
+  private Iterator<FileSplit> splitIter;
+  private FileSplit currentSplit;
+
+  public FileScanFramework(FileScanBuilder builder) {
+    super(builder);
+    assert builder.files != null;
+    assert builder.fsConf != null;
+  }
 
-  public FileScanFramework(List<SchemaPath> projection,
-      List<? extends FileWork> files,
-      Configuration fsConf,
-      FileReaderFactory readerCreator) {
-    super(projection, files, fsConf);
-    this.readerCreator = readerCreator;
+  public FileScanBuilder options() {
+    return (FileScanBuilder) builder;
   }
 
   @Override
-  protected ManagedReader<FileSchemaNegotiator> newReader(FileSplit split) throws ExecutionSetupException {
-    return readerCreator.makeBatchReader(dfs, split);
+  protected void configure() {
+    super.configure();
+    FileScanBuilder options = options();
+
+    // Create the Drill file system.
+
+    try {
+      dfs = context.newFileSystem(options.fsConf);
+    } catch (IOException e) {
+      throw UserException.dataReadError(e)
+        .addContext("Failed to create FileSystem")
+        .build(logger);
+    }
+
+    // Prepare the list of files. We need the list of paths up
+    // front to compute the maximum partition. Then, we need to
+    // iterate over the splits to create readers on demand.
+
+    List<Path> paths = new ArrayList<>();
+    for (FileWork work : options.files) {
+      Path path = dfs.makeQualified(work.getPath());
+      paths.add(path);
+      FileSplit split = new FileSplit(path, work.getStart(), work.getLength(), new String[]{""});
+      spilts.add(split);
+    }
+    splitIter = spilts.iterator();
+
+    // Create the metadata manager to handle file metadata columns
+    // (so-called implicit columns and partition columns.)
+
+    options.metadataOptions().setFiles(paths);
+    metadataManager = new FileMetadataManager(
+        context.getFragmentContext().getOptions(),
+        options.metadataOptions());
+    builder.withMetadata(metadataManager);
+  }
+
+  protected FileSplit nextSplit() {
+    if (! splitIter.hasNext()) {
+      currentSplit = null;
+      return null;
+    }
+    currentSplit = splitIter.next();
+
+    // Tell the metadata manager about the current file so it can
+    // populate the metadata columns, if requested.
+
+    metadataManager.startFile(currentSplit.getPath());
+    return currentSplit;
   }
 
   @Override
-  public boolean openReader(ShimBatchReader<FileSchemaNegotiator> shim, ManagedReader<FileSchemaNegotiator> reader) {
-    return reader.open(
-        new FileSchemaNegotiatorImpl(this, shim));
+  protected SchemaNegotiatorImpl newNegotiator() {
+    return new FileSchemaNegotiatorImpl(this);
+  }
+
+  @Override
+  public boolean open(ShimBatchReader shimBatchReader) {
+    try {
+      return super.open(shimBatchReader);
+    } catch (UserException e) {
+      throw e;
+    } catch (Exception e) {
+      throw UserException.executionError(e)
+        .addContext("File", currentSplit.getPath().toString())
+        .build(logger);
+    }
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/MetadataColumn.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/MetadataColumn.java
index bf13265..371aaa6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/MetadataColumn.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/MetadataColumn.java
@@ -23,10 +23,14 @@ import org.apache.drill.exec.physical.impl.scan.project.VectorSource;
 import org.apache.drill.exec.physical.impl.scan.project.ConstantColumnLoader.ConstantColumnSpec;
 import org.apache.drill.exec.record.MaterializedField;
 
+/**
+ * Resolved value for a metadata column (implicit file or partition column.) Resolution
+ * here means identifying a value for the column.
+ */
 public abstract class MetadataColumn extends ResolvedColumn implements ConstantColumnSpec {
 
-  public final MaterializedField schema;
-  public final String value;
+  private final MaterializedField schema;
+  private final String value;
 
   public MetadataColumn(String name, MajorType type, String value, VectorSource source, int sourceIndex) {
     super(source, sourceIndex);
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 9224afa..e1f9215 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
@@ -29,8 +29,6 @@ import org.apache.drill.exec.physical.impl.scan.project.VectorSource;
 
 public class PartitionColumn extends MetadataColumn {
 
-  public static final int ID = 16;
-
   protected final int partition;
 
   public PartitionColumn(String name, int partition) {
@@ -47,9 +45,6 @@ public class PartitionColumn extends MetadataColumn {
   public int partition() { return partition; }
 
   @Override
-  public int nodeType() { return ID; }
-
-  @Override
   public MetadataColumn resolve(FileMetadata fileInfo, VectorSource source, int sourceIndex) {
     return new PartitionColumn(name(), partition, fileInfo, source, sourceIndex);
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/BasicScanFramework.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/BasicScanFactory.java
similarity index 56%
rename from exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/BasicScanFramework.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/BasicScanFactory.java
index c5fe07b..d9e8847 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/BasicScanFramework.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/BasicScanFactory.java
@@ -18,41 +18,37 @@
 package org.apache.drill.exec.physical.impl.scan.framework;
 
 import java.util.Iterator;
-import java.util.List;
 
-import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.exec.physical.impl.scan.RowBatchReader;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedScanFramework.ReaderFactory;
 
 /**
- * Basic scan framework for simple non-file readers. Includes only
+ * Basic reader builder for simple non-file readers. Includes only
  * schema negotiation, but no implicit columns. Readers are assumed
  * to be created ahead of time and passed into the framework
  * in the constructor.
+ * <p>
+ * This form is designed to simplify conversion of existing readers.
+ * While it provides a simple first step, readers should perform
+ * further conversion to create readers on the fly rather than
+ * up front.
  */
 
-public class BasicScanFramework extends AbstractScanFramework<SchemaNegotiator> {
+public class BasicScanFactory implements ReaderFactory {
 
-  private Iterator<ManagedReader<SchemaNegotiator>> iterator;
+  private final Iterator<ManagedReader<? extends SchemaNegotiator>> iterator;
 
-  public BasicScanFramework(List<SchemaPath> projection,
-      Iterator<ManagedReader<SchemaNegotiator>> iterator) {
-    super(projection);
+  public BasicScanFactory(Iterator<ManagedReader<? extends SchemaNegotiator>> iterator) {
     this.iterator = iterator;
   }
 
   @Override
-  public RowBatchReader nextReader() {
+  public void bind(ManagedScanFramework framework) { }
+
+  @Override
+  public ManagedReader<? extends SchemaNegotiator> next() {
     if (! iterator.hasNext()) {
       return null;
     }
-    ManagedReader<SchemaNegotiator> reader = iterator.next();
-    return new ShimBatchReader<SchemaNegotiator>(this, reader);
-  }
-
-  @Override
-  public boolean openReader(ShimBatchReader<SchemaNegotiator> shim,
-      ManagedReader<SchemaNegotiator> reader) {
-    SchemaNegotiatorImpl schemaNegotiator = new SchemaNegotiatorImpl(this, shim);
-    return reader.open(schemaNegotiator);
+    return iterator.next();
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/ManagedReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/ManagedReader.java
index b57f78d..a9df2e0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/ManagedReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/ManagedReader.java
@@ -42,7 +42,7 @@ package org.apache.drill.exec.physical.impl.scan.framework;
  * result set loader obtained above. The scanner framework handles details of
  * tracking version changes, handling overflow, limiting record counts, and
  * so on. Return <tt>true</tt> to indicate a batch is available, <tt>false</tt>
- * to indicate EOF. The first call to <tt>next()</tt> can return <t>false</tt>
+ * to indicate EOF. The first call to <tt>next()</tt> can return <tt>false</tt>
  * if the data source has no rows.</li>
  * <li>{@link #close()}: called to release resources. May be called before
  * <tt>next()</tt> returns </tt>false</tt>.</li>
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/AbstractScanFramework.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/ManagedScanFramework.java
similarity index 63%
rename from exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/AbstractScanFramework.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/ManagedScanFramework.java
index d285261..394e2d9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/AbstractScanFramework.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/ManagedScanFramework.java
@@ -17,27 +17,25 @@
  */
 package org.apache.drill.exec.physical.impl.scan.framework;
 
-import java.util.List;
-
-import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.physical.impl.scan.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;
 
 /**
  * Basic scan framework for a "managed" reader which uses the scan schema
  * mechanisms encapsulated in the scan schema orchestrator. Handles binding
  * scan events to the scan orchestrator so that the scan schema is evolved
- * as the scan progresses. Subclasses are responsible for creating the actual
- * reader, which requires a framework-specific schema negotiator to be passed
- * to the reader.
+ * as the scan progresses. Readers are created and managed via a reader
+ * factory class unique to each type of scan. The reader factory also provides
+ * the scan-specific schema negotiator to be passed to the reader.
  * <p>
  * This framework is a bridge between operator logic and the scan projection
- * internals. It gathers scan-specific options, then sets
- * them on the scan orchestrator at the right time. By abstracting out this
- * plumbing, a scan batch creator simply chooses the proper framework, passes
- * config options, and implements the matching "managed reader". All details
+ * internals. It gathers scan-specific options in a builder abstraction, then
+ * passes them on the scan orchestrator at the right time. By abstracting out this
+ * plumbing, a scan batch creator simply chooses the proper framework builder, passes
+ * config options, and implements the matching "managed reader" and factory. All details
  * of setup, projection, and so on are handled by the framework and the components
  * that the framework builds upon.
  *
@@ -45,17 +43,13 @@ import org.apache.drill.exec.physical.impl.scan.project.ScanSchemaOrchestrator;
  *
  * At this basic level, a scan framework requires just a few simple inputs:
  * <ul>
- * <li>The projection list provided by the physical operator definition. This
- * list identifies the set of "output" columns whih this framework is obliged
- * to produce.</li>
+ * <li>The options defined by the scan projection framework such as the
+ * projection list.</li>
+ * <li>A reader factory to create a reader for each of the files or blocks
+ * to be scanned. (Readers are expected to be created one-by-one as files
+ * are read.)</li>
  * <li>The operator context which provides access to a memory allocator and
  * other plumbing items.</li>
- * <li>A method to create a reader for each of the files or blocks
- * defined above. (Readers are created one-by-one as files are read.)</li>
- * <li>The data type to use for projected columns which the reader cannot
- * provide. (Drill allows such columns and fills in null values: traditionally
- * nullable Int, but customizable here.)
- * <li>Various other options.</li>
  * </ul>
  *
  * <h4>Orchestration</h4>
@@ -65,7 +59,7 @@ import org.apache.drill.exec.physical.impl.scan.project.ScanSchemaOrchestrator;
  * reader (created via the factory class) differs from one type of file to
  * another.
  * <p>
- * The framework achieves the work described below= by composing a large
+ * The framework achieves the work described below by composing a large
  * set of detailed classes, each of which performs some specific task. This
  * structure leaves the reader to simply infer schema and read data.
  * <p>
@@ -80,6 +74,8 @@ import org.apache.drill.exec.physical.impl.scan.project.ScanSchemaOrchestrator;
  * data is decoded: all that is encapsulated in the reader. The only real
  * Interaction between the reader and the framework is:
  * <ul>
+ * <li>The reader factory creates a reader and the corresponding schema
+ * negotiator.</li>
  * <li>The reader "negotiates" a schema with the framework. The framework
  * knows the projection list from the query plan, knows something about
  * data types (whether a column should be scalar, a map or an array), and
@@ -113,48 +109,59 @@ import org.apache.drill.exec.physical.impl.scan.project.ScanSchemaOrchestrator;
  * </ul>
  */
 
-public abstract class AbstractScanFramework<T extends SchemaNegotiator> implements ScanOperatorEvents {
+public class ManagedScanFramework implements ScanOperatorEvents {
 
-  // Inputs
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ManagedScanFramework.class);
 
-  protected final List<SchemaPath> projection;
-  protected MajorType nullType;
-  protected int maxBatchRowCount;
-  protected int maxBatchByteCount;
-  protected OperatorContext context;
+  /**
+   * Creates a batch reader on demand. Unlike earlier versions of Drill,
+   * this framework creates readers one by one, when they are needed.
+   * Doing so avoids excessive resource demands that come from creating
+   * potentially thousands of readers up front.
+   * <p>
+   * The reader itself is unique to each file type. This interface
+   * provides a common interface that this framework can use to create the
+   * file-specific reader on demand.
+   * <p>
+   * Also manages opening the reader using a scan-specific schema
+   * negotiator.
+   */
 
-  // Internal state
+  public interface ReaderFactory {
+    void bind(ManagedScanFramework framework);
+    ManagedReader<? extends SchemaNegotiator> next();
+  }
 
-  protected ScanSchemaOrchestrator scanOrchestrator;
+  public static class ScanFrameworkBuilder extends ScanOrchestratorBuilder {
+    protected ReaderFactory readerFactory;
 
-  public AbstractScanFramework(List<SchemaPath> projection) {
-    this.projection = projection;
+    public void setReaderFactory(ReaderFactory readerFactory) {
+      this.readerFactory = readerFactory;
+    }
   }
 
-  /**
-   * Specify the type to use for projected columns that do not
-   * match any data source columns. Defaults to nullable int.
-   */
+  // Inputs
 
-  public void setNullType(MajorType type) {
-    this.nullType = type;
-  }
+  protected final ScanFrameworkBuilder builder;
+  protected final ReaderFactory readerFactory;
+  protected OperatorContext context;
 
-  public void setMaxRowCount(int rowCount) {
-    maxBatchRowCount = rowCount;
-  }
+  // Internal state
+
+  protected ScanSchemaOrchestrator scanOrchestrator;
 
-  public void setMaxBatchByteCount(int byteCount) {
-    maxBatchByteCount = byteCount;
+  public ManagedScanFramework(ScanFrameworkBuilder builder) {
+    this.builder = builder;
+    readerFactory = builder.readerFactory;
+    assert readerFactory != null;
   }
 
   @Override
   public void bind(OperatorContext context) {
     this.context = context;
-    scanOrchestrator = new ScanSchemaOrchestrator(context.getAllocator());
     configure();
-    assert projection != null;
-    scanOrchestrator.build(projection);
+    scanOrchestrator = new ScanSchemaOrchestrator(context.getAllocator(), builder);
+    readerFactory.bind(this);
   }
 
   public OperatorContext context() { return context; }
@@ -163,22 +170,24 @@ public abstract class AbstractScanFramework<T extends SchemaNegotiator> implemen
     return scanOrchestrator;
   }
 
-  protected void configure() {
+  protected void configure() { }
 
-    // Pass along config options if set.
+  @Override
+  public RowBatchReader nextReader() {
+    ManagedReader<? extends SchemaNegotiator> reader = readerFactory.next();
+    return reader == null ? null : new ShimBatchReader(this, reader);
+  }
 
-    if (maxBatchRowCount > 0) {
-      scanOrchestrator.setBatchRecordLimit(maxBatchRowCount);
-    }
-    if (maxBatchByteCount > 0) {
-      scanOrchestrator.setBatchByteLimit(maxBatchByteCount);
-    }
-    if (nullType != null) {
-      scanOrchestrator.setNullType(nullType);
-    }
+  protected SchemaNegotiatorImpl newNegotiator() {
+    return new SchemaNegotiatorImpl(this);
   }
 
-  public abstract boolean openReader(ShimBatchReader<T> shim, ManagedReader<T> reader);
+  @SuppressWarnings("unchecked")
+  public boolean open(ShimBatchReader shimBatchReader) {
+    SchemaNegotiatorImpl schemaNegotiator = newNegotiator();
+    schemaNegotiator.bind(shimBatchReader);
+    return ((ManagedReader<SchemaNegotiator>) shimBatchReader.reader()).open(schemaNegotiator);
+  }
 
   @Override
   public void close() {
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 0841049..a8e02cd 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
@@ -50,20 +50,27 @@ import org.apache.drill.exec.vector.ValueVector;
 
 public class SchemaNegotiatorImpl implements SchemaNegotiator {
 
-  protected final AbstractScanFramework<?> basicFramework;
-  private final ShimBatchReader<? extends SchemaNegotiator> shim;
+  public interface NegotiatorListener {
+    ResultSetLoader build(SchemaNegotiatorImpl schemaNegotiator);
+  }
+
+  protected final ManagedScanFramework framework;
+  private NegotiatorListener listener;
   protected TupleMetadata tableSchema;
   protected boolean isSchemaComplete;
   protected int batchSize = ValueVector.MAX_ROW_COUNT;
 
-  public SchemaNegotiatorImpl(AbstractScanFramework<?> framework, ShimBatchReader<? extends SchemaNegotiator> shim) {
-    basicFramework = framework;
-    this.shim = shim;
+  public SchemaNegotiatorImpl(ManagedScanFramework framework) {
+    this.framework = framework;
+  }
+
+  public void bind(NegotiatorListener listener) {
+    this.listener = listener;
   }
 
   @Override
   public OperatorContext context() {
-    return basicFramework.context();
+    return framework.context();
   }
 
   @Override
@@ -92,12 +99,12 @@ public class SchemaNegotiatorImpl implements SchemaNegotiator {
 
     // Build and return the result set loader to be used by the reader.
 
-    return shim.build(this);
+    return listener.build(this);
   }
 
   @Override
   public boolean isProjectionEmpty() {
-    return basicFramework.scanOrchestrator().isProjectNone();
+    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 a97b329..d3407d3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/ShimBatchReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/framework/ShimBatchReader.java
@@ -19,6 +19,7 @@ package org.apache.drill.exec.physical.impl.scan.framework;
 
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.physical.impl.scan.RowBatchReader;
+import org.apache.drill.exec.physical.impl.scan.framework.SchemaNegotiatorImpl.NegotiatorListener;
 import org.apache.drill.exec.physical.impl.scan.project.ReaderSchemaOrchestrator;
 import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
 import org.apache.drill.exec.record.VectorContainer;
@@ -37,12 +38,12 @@ import org.apache.drill.exec.record.VectorContainer;
  * of solutions as needed for different readers.
  */
 
-public class ShimBatchReader<T extends SchemaNegotiator> implements RowBatchReader {
+public class ShimBatchReader implements RowBatchReader, NegotiatorListener {
 
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ShimBatchReader.class);
 
-  protected final AbstractScanFramework<T> manager;
-  protected final ManagedReader<T> reader;
+  protected final ManagedScanFramework framework;
+  protected final ManagedReader<? extends SchemaNegotiator> reader;
   protected final ReaderSchemaOrchestrator readerOrchestrator;
   protected SchemaNegotiatorImpl schemaNegotiator;
   protected ResultSetLoader tableLoader;
@@ -54,8 +55,8 @@ public class ShimBatchReader<T extends SchemaNegotiator> implements RowBatchRead
 
   private boolean eof;
 
-  public ShimBatchReader(AbstractScanFramework<T> manager, ManagedReader<T> reader) {
-    this.manager = manager;
+  public ShimBatchReader(ManagedScanFramework manager, ManagedReader<? extends SchemaNegotiator> reader) {
+    this.framework = manager;
     this.reader = reader;
     readerOrchestrator = manager.scanOrchestrator().startReader();
   }
@@ -65,12 +66,14 @@ public class ShimBatchReader<T extends SchemaNegotiator> implements RowBatchRead
     return reader.getClass().getSimpleName();
   }
 
+  public ManagedReader<? extends SchemaNegotiator> reader() { return reader; }
+
   @Override
   public boolean open() {
 
     // Build and return the result set loader to be used by the reader.
 
-    if (! manager.openReader(this, reader)) {
+    if (! framework.open(this)) {
 
       // If we had a soft failure, then there should be no schema.
       // The reader should not have negotiated one. Not a huge
@@ -149,8 +152,8 @@ public class ShimBatchReader<T extends SchemaNegotiator> implements RowBatchRead
     // Output should be defined only if vector schema has
     // been defined.
 
-    if (manager.scanOrchestrator().hasSchema()) {
-      return manager.scanOrchestrator().output();
+    if (framework.scanOrchestrator().hasSchema()) {
+      return framework.scanOrchestrator().output();
     } else {
       return null;
     }
@@ -176,7 +179,7 @@ public class ShimBatchReader<T extends SchemaNegotiator> implements RowBatchRead
     // closes the table loader, so we don't close the table loader
     // here.
 
-    manager.scanOrchestrator().closeReader();
+    framework.scanOrchestrator().closeReader();
 
     // Throw any exceptions.
 
@@ -190,6 +193,7 @@ public class ShimBatchReader<T extends SchemaNegotiator> implements RowBatchRead
     return tableLoader.schemaVersion();
   }
 
+  @Override
   public ResultSetLoader build(SchemaNegotiatorImpl schemaNegotiator) {
     this.schemaNegotiator = schemaNegotiator;
     readerOrchestrator.setBatchSize(schemaNegotiator.batchSize);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/UnresolvedColumn.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/AbstractUnresolvedColumn.java
similarity index 67%
rename from exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/UnresolvedColumn.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/AbstractUnresolvedColumn.java
index 2dfa9c4..0bbae00 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/UnresolvedColumn.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/AbstractUnresolvedColumn.java
@@ -18,16 +18,40 @@
 package org.apache.drill.exec.physical.impl.scan.project;
 
 import org.apache.drill.exec.physical.rowSet.project.RequestedTuple.RequestedColumn;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
 
 /**
  * Represents a projected column that has not yet been bound to a
  * table column, special column or a null column. Once bound, this
  * column projection is replaced with the detailed binding.
  */
-public class UnresolvedColumn implements ColumnProjection {
+public abstract class AbstractUnresolvedColumn implements ColumnProjection {
 
-  public static final int WILDCARD = 1;
-  public static final int UNRESOLVED = 2;
+  public static class UnresolvedColumn extends AbstractUnresolvedColumn {
+
+    public UnresolvedColumn(RequestedColumn inCol) {
+      super(inCol);
+    }
+  }
+
+  public static class UnresolvedWildcardColumn extends AbstractUnresolvedColumn {
+
+    public UnresolvedWildcardColumn(RequestedColumn inCol) {
+      super(inCol);
+    }
+  }
+
+  public static class UnresolvedSchemaColumn extends AbstractUnresolvedColumn {
+
+    private final ColumnMetadata colDefn;
+
+    public UnresolvedSchemaColumn(RequestedColumn inCol, ColumnMetadata colDefn) {
+      super(inCol);
+      this.colDefn = colDefn;
+    }
+
+    public ColumnMetadata metadata() { return colDefn; }
+  }
 
   /**
    * The original physical plan column to which this output column
@@ -36,17 +60,12 @@ public class UnresolvedColumn implements ColumnProjection {
    */
 
   protected final RequestedColumn inCol;
-  private final int id;
 
-  public UnresolvedColumn(RequestedColumn inCol, int id) {
+  public AbstractUnresolvedColumn(RequestedColumn inCol) {
     this.inCol = inCol;
-    this.id = id;
   }
 
   @Override
-  public int nodeType() { return id; }
-
-  @Override
   public String name() { return inCol.name(); }
 
   public RequestedColumn element() { return inCol; }
@@ -58,7 +77,7 @@ public class UnresolvedColumn implements ColumnProjection {
       .append("[")
       .append(getClass().getSimpleName())
       .append(" type=")
-      .append(id == WILDCARD ? "wildcard" : "column");
+      .append(getClass().getSimpleName());
     if (inCol != null) {
       buf
         .append(", incol=")
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ColumnProjection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ColumnProjection.java
index 0540d0b..bcde240 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ColumnProjection.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ColumnProjection.java
@@ -38,9 +38,6 @@ package org.apache.drill.exec.physical.impl.scan.project;
 
 public interface ColumnProjection {
 
-  public static final int FRAMEWORK_BASE_ID = 100;
-  public static final int READER_BASE_ID = 200;
-
   /**
    * The name of the column as it appears in the output
    * row (record batch.)
@@ -48,30 +45,4 @@ public interface ColumnProjection {
    * @return the output column name
    */
   String name();
-
-  /**
-   * A node type unique to each node. Nodes defined in this package
-   * use IDs less than {@link #FRAMEWORK_BASE_ID}. Nodes defined by
-   * frameworks (for file metadata columns or for other special
-   * columns) start numbering with {@link #FRAMEWORK_BASE_ID}. Readers
-   * may need their own specialized nodes, which must use IDs starting
-   * with {@link #READER_BASE_ID}.
-   * <p>
-   * This system solves two problems:
-   * <ol>
-   * <li>Provides an efficient way for each mechanism to recognize its
-   * own nodes without using <code>instance of</code>.</li>
-   * <li>Allows for frameworks and readers to be added without changing
-   * any base enum. This works because every instance of this mechanism
-   * sees only the base nodes, those from a single framework and those
-   * from a single reader; there is no need for a universal ID registry.
-   * Two frameworks can use identical IDs because they never mix.
-   * Similarly, two readers can use the same IDs because Drill does not
-   * allow a single scan operator to use two different kinds of readers.
-   * </li>
-   * </ol>
-   * @return the numeric ID for this node, used for each layer to
-   * recognize its own nodes
-   */
-  int nodeType();
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ExplicitSchemaProjection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ExplicitSchemaProjection.java
index c0bcfa3..16f0035 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ExplicitSchemaProjection.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ExplicitSchemaProjection.java
@@ -21,6 +21,7 @@ import java.util.List;
 
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.project.AbstractUnresolvedColumn.UnresolvedColumn;
 import org.apache.drill.exec.physical.rowSet.project.RequestedTuple;
 import org.apache.drill.exec.physical.rowSet.project.RequestedTuple.RequestedColumn;
 import org.apache.drill.exec.record.MaterializedField;
@@ -40,37 +41,37 @@ import org.apache.drill.exec.record.metadata.TupleMetadata;
  * unmatched projections.
  */
 
-public class ExplicitSchemaProjection extends SchemaLevelProjection {
+public class ExplicitSchemaProjection extends ReaderLevelProjection {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExplicitSchemaProjection.class);
 
   public ExplicitSchemaProjection(ScanLevelProjection scanProj,
-      TupleMetadata tableSchema,
+      TupleMetadata readerSchema,
       ResolvedTuple rootTuple,
-      List<SchemaProjectionResolver> resolvers) {
+      List<ReaderProjectionResolver> resolvers) {
     super(resolvers);
-    resolveRootTuple(scanProj, rootTuple, tableSchema);
+    resolveRootTuple(scanProj, rootTuple, readerSchema);
   }
 
   private void resolveRootTuple(ScanLevelProjection scanProj,
       ResolvedTuple rootTuple,
-      TupleMetadata tableSchema) {
+      TupleMetadata readerSchema) {
     for (ColumnProjection col : scanProj.columns()) {
-      if (col.nodeType() == UnresolvedColumn.UNRESOLVED) {
-        resolveColumn(rootTuple, ((UnresolvedColumn) col).element(), tableSchema);
+      if (col instanceof UnresolvedColumn) {
+        resolveColumn(rootTuple, ((UnresolvedColumn) col).element(), readerSchema);
       } else {
-        resolveSpecial(rootTuple, col, tableSchema);
+        resolveSpecial(rootTuple, col, readerSchema);
       }
     }
   }
 
   private void resolveColumn(ResolvedTuple outputTuple,
-      RequestedColumn inputCol, TupleMetadata tableSchema) {
-    int tableColIndex = tableSchema.index(inputCol.name());
+      RequestedColumn inputCol, TupleMetadata readerSchema) {
+    int tableColIndex = readerSchema.index(inputCol.name());
     if (tableColIndex == -1) {
       resolveNullColumn(outputTuple, inputCol);
     } else {
       resolveTableColumn(outputTuple, inputCol,
-          tableSchema.metadata(tableColIndex),
+          readerSchema.metadata(tableColIndex),
           tableColIndex);
     }
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/MetadataManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/MetadataManager.java
index fe1e024..ddd2660 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/MetadataManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/MetadataManager.java
@@ -18,7 +18,7 @@
 package org.apache.drill.exec.physical.impl.scan.project;
 
 import org.apache.drill.exec.physical.impl.scan.project.ScanLevelProjection.ScanProjectionParser;
-import org.apache.drill.exec.physical.impl.scan.project.SchemaLevelProjection.SchemaProjectionResolver;
+import org.apache.drill.exec.physical.impl.scan.project.ReaderLevelProjection.ReaderProjectionResolver;
 import org.apache.drill.exec.physical.rowSet.ResultVectorCache;
 
 /**
@@ -45,7 +45,7 @@ public interface MetadataManager {
 
   ScanProjectionParser projectionParser();
 
-  SchemaProjectionResolver resolver();
+  ReaderProjectionResolver resolver();
 
   /**
    * Define (materialize) the columns which this manager
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/NoOpMetadataManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/NoOpMetadataManager.java
index ec95de8..7f2197d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/NoOpMetadataManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/NoOpMetadataManager.java
@@ -18,7 +18,7 @@
 package org.apache.drill.exec.physical.impl.scan.project;
 
 import org.apache.drill.exec.physical.impl.scan.project.ScanLevelProjection.ScanProjectionParser;
-import org.apache.drill.exec.physical.impl.scan.project.SchemaLevelProjection.SchemaProjectionResolver;
+import org.apache.drill.exec.physical.impl.scan.project.ReaderLevelProjection.ReaderProjectionResolver;
 import org.apache.drill.exec.physical.rowSet.ResultVectorCache;
 
 /**
@@ -36,7 +36,7 @@ public class NoOpMetadataManager implements MetadataManager {
   public ScanProjectionParser projectionParser() { return null; }
 
   @Override
-  public SchemaProjectionResolver resolver() {
+  public ReaderProjectionResolver resolver() {
     // The resolver is requested only for user-defined metadata
     // managers, not for this default, no-op version. If this
     // method is called, something is amiss with the default
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/NullColumnBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/NullColumnBuilder.java
index 7e9d2fd..311f44e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/NullColumnBuilder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/NullColumnBuilder.java
@@ -24,6 +24,8 @@ import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.exec.physical.impl.scan.project.NullColumnLoader.NullColumnSpec;
 import org.apache.drill.exec.physical.rowSet.ResultVectorCache;
 import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.vector.ValueVector;
 
 import org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTesting;
@@ -36,6 +38,31 @@ import org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTes
 
 public class NullColumnBuilder implements VectorSource {
 
+  public static class NullBuilderBuilder {
+    protected MajorType nullType;
+    protected boolean allowRequiredNullColumns;
+    protected TupleMetadata outputSchema;
+
+    public NullBuilderBuilder setNullType(MajorType nullType) {
+      this.nullType = nullType;
+      return this;
+    }
+
+    public NullBuilderBuilder allowRequiredNullColumns(boolean flag) {
+      allowRequiredNullColumns = flag;
+      return this;
+    }
+
+    public NullBuilderBuilder setOutputSchema(TupleMetadata outputSchema) {
+      this.outputSchema = outputSchema;
+      return this;
+    }
+
+    public NullColumnBuilder build() {
+      return new NullColumnBuilder(this);
+    }
+  }
+
   /**
    * Creates null columns if needed.
    */
@@ -43,6 +70,7 @@ public class NullColumnBuilder implements VectorSource {
   protected final List<NullColumnSpec> nullCols = new ArrayList<>();
   private NullColumnLoader nullColumnLoader;
   private VectorContainer outputContainer;
+  protected TupleMetadata outputSchema;
 
   /**
    * The reader-specified null type if other than the default.
@@ -51,22 +79,76 @@ public class NullColumnBuilder implements VectorSource {
   private final MajorType nullType;
   private final boolean allowRequiredNullColumns;
 
-  public NullColumnBuilder(
-      MajorType nullType, boolean allowRequiredNullColumns) {
-    this.nullType = nullType;
-    this.allowRequiredNullColumns = allowRequiredNullColumns;
+  public NullColumnBuilder(NullBuilderBuilder builder) {
+    this.nullType = builder.nullType;
+    this.allowRequiredNullColumns = builder.allowRequiredNullColumns;
+    this.outputSchema = builder.outputSchema;
   }
 
-  public NullColumnBuilder newChild() {
-    return new NullColumnBuilder(nullType, allowRequiredNullColumns);
+  public NullColumnBuilder newChild(String mapName) {
+    NullBuilderBuilder builder = new NullBuilderBuilder()
+        .setNullType(nullType)
+        .allowRequiredNullColumns(allowRequiredNullColumns);
+
+    // Pass along the schema of the child map if 1) we have an output schema,
+    // 2) the column is defined in that schema, and 3) the column is a map.
+    if (outputSchema != null) {
+      ColumnMetadata colSchema = outputSchema.metadata(mapName);
+      if (colSchema != null) {
+        builder.setOutputSchema(colSchema.mapSchema());
+      }
+    }
+    return builder.build();
   }
 
   public ResolvedNullColumn add(String name) {
     return add(name, null);
   }
 
+  public ResolvedNullColumn add(ColumnMetadata colDefn) {
+    final ResolvedNullColumn col = new ResolvedNullColumn(
+        colDefn, this, nullCols.size());
+    nullCols.add(col);
+    return col;
+  }
+
   public ResolvedNullColumn add(String name, MajorType type) {
-    final ResolvedNullColumn col = new ResolvedNullColumn(name, type, this, nullCols.size());
+
+    // If type is provided, use it. (Used during schema smoothing.)
+    // Else if there is an output schema, and the column appears in
+    // that schema, use that type.
+
+    ResolvedNullColumn col = null;
+    if (outputSchema != null) {
+      ColumnMetadata outputCol = outputSchema.metadata(name);
+      if (outputCol != null) {
+        if (type == null) {
+
+          // No preferred type, so no conflict
+
+          col = new ResolvedNullColumn(outputCol, this, nullCols.size());
+        } else if (type.getMinorType() != outputCol.type()) {
+
+          // Conflict in type, can't use default value.
+
+        } else if (type.getMode() != outputCol.mode()) {
+
+          // Conflict in mode. Use the specified name and type, but
+          // use the default value from the output schema.
+
+          col = new ResolvedNullColumn(name, type,
+              outputCol.decodeDefaultValue(), this, nullCols.size());
+        } else {
+
+          // Type and modes matches, just the output column
+
+          col = new ResolvedNullColumn(outputCol, this, nullCols.size());
+        }
+      }
+    }
+    if (col == null) {
+      col = new ResolvedNullColumn(name, type, null, this, nullCols.size());
+    }
     nullCols.add(col);
     return col;
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/NullColumnLoader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/NullColumnLoader.java
index 0c36cb9..9e48d62 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/NullColumnLoader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/NullColumnLoader.java
@@ -62,6 +62,7 @@ public class NullColumnLoader extends StaticColumnLoader {
     String name();
     MajorType type();
     void setType(MajorType type);
+    Object defaultValue();
   }
 
   public static final MajorType DEFAULT_NULL_TYPE = MajorType.newBuilder()
@@ -71,11 +72,14 @@ public class NullColumnLoader extends StaticColumnLoader {
 
   private final MajorType nullType;
   private final boolean allowRequired;
-  private final boolean isArray[];
+  private final List<? extends NullColumnSpec> colDefns;
+  private final int colsWithDefaultValues[];
+  private final Object defaultValues[];
 
   public NullColumnLoader(ResultVectorCache vectorCache, List<? extends NullColumnSpec> defns,
       MajorType nullType, boolean allowRequired) {
     super(vectorCache);
+    this.colDefns = defns;
 
     // Normally, null columns must be optional or arrays. However
     // we allow required columns either if the client requests it,
@@ -98,12 +102,32 @@ public class NullColumnLoader extends StaticColumnLoader {
     // Populate the loader schema from that provided
 
     RowSetLoader schema = loader.writer();
-    isArray = new boolean[defns.size()];
+    int defaultCount = 0;
     for (int i = 0; i < defns.size(); i++) {
       NullColumnSpec defn = defns.get(i);
       MaterializedField colSchema = selectType(defn);
-      isArray[i] = colSchema.getDataMode() == DataMode.REPEATED;
       schema.addColumn(colSchema);
+      if (defn.defaultValue() != null) {
+        defaultCount++;
+      }
+    }
+
+    // Setup default values, if any
+
+    if (defaultCount == 0) {
+      colsWithDefaultValues = null;
+      defaultValues = null;
+      return;
+    }
+    colsWithDefaultValues = new int[defaultCount];
+    defaultValues = new Object[defaultCount];
+    int defIndex = 0;
+    for (int i = 0; i < defns.size(); i++) {
+      NullColumnSpec defn = defns.get(i);
+      if (defn.defaultValue() != null) {
+        colsWithDefaultValues[defIndex] = i;
+        defaultValues[defIndex++] = defn.defaultValue();
+      }
     }
   }
 
@@ -126,7 +150,7 @@ public class NullColumnLoader extends StaticColumnLoader {
     if (type == null) {
       type = defn.type();
     }
-    if (type != null && ! allowRequired && type.getMode() == DataMode.REQUIRED) {
+    if (type != null && ! allowRequired && type.getMode() == DataMode.REQUIRED && defn.defaultValue() == null) {
 
       // Type was found in the vector cache and the type is required.
       // The client determines whether to map required types to optional.
@@ -164,7 +188,20 @@ public class NullColumnLoader extends StaticColumnLoader {
   @Override
   public VectorContainer load(int rowCount) {
     loader.startBatch();
-    loader.skipRows(rowCount);
+    if (colsWithDefaultValues == null) {
+      loader.skipRows(rowCount);
+    } else {
+     // At least one column has a default value. Set values
+      // for all columns. Any null values are a no-op.
+      RowSetLoader writer = loader.writer();
+      for (int i = 0; i < rowCount; i++) {
+        writer.start();
+        for (int j = 0; j < colsWithDefaultValues.length; j++) {
+          writer.scalar(colsWithDefaultValues[j]).setValue(defaultValues[j]);
+        }
+        writer.save();
+      }
+    }
     return loader.harvest();
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/SchemaLevelProjection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ReaderLevelProjection.java
similarity index 87%
rename from exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/SchemaLevelProjection.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ReaderLevelProjection.java
index a756114..8b46d88 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/SchemaLevelProjection.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ReaderLevelProjection.java
@@ -59,41 +59,41 @@ import org.apache.drill.exec.record.metadata.TupleMetadata;
  * evolved
  */
 
-public class SchemaLevelProjection {
+public class ReaderLevelProjection {
 
   /**
-   * Schema-level projection is customizable. Implement this interface, and
+   * Reader-level projection is customizable. Implement this interface, and
    * add an instance to the scan orchestrator, to perform custom mappings
    * from unresolved columns (perhaps of an extension-specified type) to
    * final projected columns. The metadata manager, for example, implements
    * this interface to map metadata columns.
    */
 
-  public interface SchemaProjectionResolver {
+  public interface ReaderProjectionResolver {
     void startResolution();
     boolean resolveColumn(ColumnProjection col, ResolvedTuple tuple,
         TupleMetadata tableSchema);
   }
 
-  protected final List<SchemaProjectionResolver> resolvers;
+  protected final List<ReaderProjectionResolver> resolvers;
 
-  protected SchemaLevelProjection(
-        List<SchemaProjectionResolver> resolvers) {
+  protected ReaderLevelProjection(
+        List<ReaderProjectionResolver> resolvers) {
     this.resolvers = resolvers == null ? new ArrayList<>() : resolvers;
-    for (SchemaProjectionResolver resolver : resolvers) {
+    for (ReaderProjectionResolver resolver : resolvers) {
       resolver.startResolution();
     }
   }
 
   protected void resolveSpecial(ResolvedTuple rootOutputTuple, ColumnProjection col,
       TupleMetadata tableSchema) {
-    for (SchemaProjectionResolver resolver : resolvers) {
+    for (ReaderProjectionResolver resolver : resolvers) {
       if (resolver.resolveColumn(col, rootOutputTuple, tableSchema)) {
         return;
       }
     }
     throw new IllegalStateException(
-        String.format("No resolver for column `%s` of type %d",
-            col.name(), col.nodeType()));
+        String.format("No resolver for column `%s` of type %s",
+            col.name(), col.getClass().getSimpleName()));
   }
 }
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 029b6a0..5a2c651 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
@@ -17,6 +17,7 @@
  */
 package org.apache.drill.exec.physical.impl.scan.project;
 
+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.rowSet.ResultSetLoader;
 import org.apache.drill.exec.physical.rowSet.impl.OptionBuilder;
@@ -34,7 +35,7 @@ import org.apache.drill.exec.vector.ValueVector;
 public class ReaderSchemaOrchestrator implements VectorSource {
 
   private final ScanSchemaOrchestrator scanOrchestrator;
-  private int readerBatchSize;
+  private int readerBatchSize = ScanSchemaOrchestrator.MAX_BATCH_ROW_COUNT;
   private ResultSetLoaderImpl tableLoader;
   private int prevTableSchemaVersion = -1;
 
@@ -50,20 +51,21 @@ public class ReaderSchemaOrchestrator implements VectorSource {
 
   public ReaderSchemaOrchestrator(ScanSchemaOrchestrator scanSchemaOrchestrator) {
     scanOrchestrator = scanSchemaOrchestrator;
-    readerBatchSize = scanOrchestrator.scanBatchRecordLimit;
+    readerBatchSize = scanOrchestrator.options.scanBatchRecordLimit;
   }
 
   public void setBatchSize(int size) {
     if (size > 0) {
-      readerBatchSize = Math.min(size, scanOrchestrator.scanBatchRecordLimit);
+      readerBatchSize = size;
     }
   }
 
-  public ResultSetLoader makeTableLoader(TupleMetadata tableSchema) {
+  public ResultSetLoader makeTableLoader(TupleMetadata readerSchema) {
     OptionBuilder options = new OptionBuilder();
-    options.setRowCountLimit(readerBatchSize);
+    options.setRowCountLimit(Math.min(readerBatchSize, scanOrchestrator.options.scanBatchRecordLimit));
     options.setVectorCache(scanOrchestrator.vectorCache);
-    options.setBatchSizeLimit(scanOrchestrator.scanBatchByteLimit);
+    options.setBatchSizeLimit(scanOrchestrator.options.scanBatchByteLimit);
+    options.setSchemaTransform(scanOrchestrator.options.schemaTransformer);
 
     // Set up a selection list if available and is a subset of
     // table columns. (Only needed for non-wildcard queries.)
@@ -75,7 +77,7 @@ public class ReaderSchemaOrchestrator implements VectorSource {
     if (! scanOrchestrator.scanProj.projectAll()) {
       options.setProjectionSet(scanOrchestrator.scanProj.readerProjection());
     }
-    options.setSchema(tableSchema);
+    options.setSchema(readerSchema);
 
     // Create the table loader
 
@@ -143,13 +145,25 @@ public class ReaderSchemaOrchestrator implements VectorSource {
     // Do the table-schema level projection; the final matching
     // of projected columns to available columns.
 
-    TupleMetadata tableSchema = tableLoader.harvestSchema();
+    TupleMetadata readerSchema = tableLoader.harvestSchema();
     if (scanOrchestrator.schemaSmoother != null) {
-      doSmoothedProjection(tableSchema);
-    } else if (scanOrchestrator.scanProj.hasWildcard()) {
-      doWildcardProjection(tableSchema);
+      doSmoothedProjection(readerSchema);
     } else {
-      doExplicitProjection(tableSchema);
+      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());
+      }
     }
 
     // Combine metadata, nulls and batch data to form the final
@@ -164,8 +178,7 @@ public class ReaderSchemaOrchestrator implements VectorSource {
   }
 
   private void doSmoothedProjection(TupleMetadata tableSchema) {
-    rootTuple = new ResolvedRow(
-        new NullColumnBuilder(scanOrchestrator.nullType, scanOrchestrator.allowRequiredNullColumns));
+    rootTuple = newRootTuple();
     scanOrchestrator.schemaSmoother.resolve(tableSchema, rootTuple);
   }
 
@@ -175,9 +188,25 @@ public class ReaderSchemaOrchestrator implements VectorSource {
    */
 
   private void doWildcardProjection(TupleMetadata tableSchema) {
-    rootTuple = new ResolvedRow(null);
+    rootTuple = newRootTuple();
+    new WildcardProjection(scanOrchestrator.scanProj,
+        tableSchema, rootTuple, scanOrchestrator.options.schemaResolvers);
+  }
+
+  private void doStrictWildcardProjection(TupleMetadata tableSchema) {
+    rootTuple = newRootTuple();
     new WildcardSchemaProjection(scanOrchestrator.scanProj,
-        tableSchema, rootTuple, scanOrchestrator.schemaResolvers);
+        tableSchema, rootTuple, scanOrchestrator.options.schemaResolvers);
+  }
+
+  private ResolvedRow newRootTuple() {
+    NullBuilderBuilder nullBuilder = new NullBuilderBuilder()
+        .setNullType(scanOrchestrator.options.nullType)
+        .allowRequiredNullColumns(scanOrchestrator.options.allowRequiredNullColumns);
+    if (scanOrchestrator.options.schemaTransformer != null) {
+      nullBuilder.setOutputSchema(scanOrchestrator.options.schemaTransformer.outputSchema());
+    }
+    return new ResolvedRow(nullBuilder.build());
   }
 
   /**
@@ -190,11 +219,10 @@ public class ReaderSchemaOrchestrator implements VectorSource {
    */
 
   private void doExplicitProjection(TupleMetadata tableSchema) {
-    rootTuple = new ResolvedRow(
-        new NullColumnBuilder(scanOrchestrator.nullType, scanOrchestrator.allowRequiredNullColumns));
+    rootTuple = newRootTuple();
     new ExplicitSchemaProjection(scanOrchestrator.scanProj,
             tableSchema, rootTuple,
-            scanOrchestrator.schemaResolvers);
+            scanOrchestrator.options.schemaResolvers);
   }
 
   @Override
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 a658629..be0cb30 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
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.physical.impl.scan.project;
 
 import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
 
 /**
  * A resolved column has a name, and a specification for how to project
@@ -36,18 +37,28 @@ import org.apache.drill.exec.record.MaterializedField;
 
 public abstract class ResolvedColumn implements ColumnProjection {
 
-  public final VectorSource source;
-  public final int sourceIndex;
+  private final VectorSource source;
+  private final int sourceIndex;
+  private final ColumnMetadata outputCol;
 
   public ResolvedColumn(VectorSource source, int sourceIndex) {
     this.source = source;
     this.sourceIndex = sourceIndex;
+    outputCol = null;
+  }
+
+  public ResolvedColumn(ColumnMetadata outputCol, VectorSource source, int sourceIndex) {
+    this.source = source;
+    this.sourceIndex = sourceIndex;
+    this.outputCol = outputCol;
   }
 
   public VectorSource source() { return source; }
 
   public int sourceIndex() { return sourceIndex; }
 
+  public ColumnMetadata metadata() { return outputCol; }
+
   /**
    * Return the type of this column. Used primarily by the schema smoothing
    * mechanism.
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 4a158f7..1ba2d77 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
@@ -33,8 +33,6 @@ import org.apache.drill.exec.record.MaterializedField;
 
 public class ResolvedMapColumn extends ResolvedColumn {
 
-  public static final int ID = 17;
-
   private final MaterializedField schema;
   private final ResolvedTuple parent;
   private final ResolvedMap members;
@@ -73,9 +71,6 @@ public class ResolvedMapColumn extends ResolvedColumn {
   @Override
   public String name() { return schema.getName(); }
 
-  @Override
-  public int nodeType() { return ID; }
-
   public ResolvedTuple members() { return members; }
 
   @Override
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 7694b7e..80f9f0b 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
@@ -20,6 +20,7 @@ package org.apache.drill.exec.physical.impl.scan.project;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.exec.physical.impl.scan.project.NullColumnLoader.NullColumnSpec;
 import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
 
 /**
  * Projected column that serves as both a resolved column (provides projection
@@ -29,15 +30,24 @@ import org.apache.drill.exec.record.MaterializedField;
 
 public class ResolvedNullColumn extends ResolvedColumn implements NullColumnSpec {
 
-  public static final int ID = 4;
-
   private final String name;
   private MajorType type;
+  private Object defaultValue;
 
-  public ResolvedNullColumn(String name, MajorType type, VectorSource source, int sourceIndex) {
+  public ResolvedNullColumn(String name, MajorType type, Object defaultValue,
+      VectorSource source, int sourceIndex) {
     super(source, sourceIndex);
     this.name = name;
     this.type = type;
+    this.defaultValue = defaultValue;
+  }
+
+  public ResolvedNullColumn(ColumnMetadata colDefn,
+      VectorSource source, int sourceIndex) {
+    super(colDefn, source, sourceIndex);
+    this.name = colDefn.name();
+    this.type = colDefn.majorType();
+    this.defaultValue = colDefn.decodeDefaultValue();
   }
 
   @Override
@@ -47,11 +57,14 @@ public class ResolvedNullColumn extends ResolvedColumn implements NullColumnSpec
   public MajorType type() { return type; }
 
   @Override
-  public int nodeType() { return ID; }
-
-  @Override
   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.
 
@@ -62,4 +75,7 @@ public class ResolvedNullColumn extends ResolvedColumn implements NullColumnSpec
   public MaterializedField schema() {
     return MaterializedField.create(name, type);
   }
+
+  @Override
+  public Object defaultValue() { return defaultValue; }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ResolvedTableColumn.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ResolvedTableColumn.java
index f17802a..7df1b72 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ResolvedTableColumn.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ResolvedTableColumn.java
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.physical.impl.scan.project;
 
 import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
 
 /**
  * Column that matches one provided by the table. Provides the data type
@@ -30,10 +31,8 @@ import org.apache.drill.exec.record.MaterializedField;
 
 public class ResolvedTableColumn extends ResolvedColumn {
 
-  public static final int ID = 3;
-
-  public final String projectedName;
-  public final MaterializedField schema;
+  private final String projectedName;
+  private final MaterializedField schema;
 
   public ResolvedTableColumn(String projectedName,
       MaterializedField schema,
@@ -43,6 +42,13 @@ public class ResolvedTableColumn extends ResolvedColumn {
     this.schema = schema;
   }
 
+  public ResolvedTableColumn(ColumnMetadata outputCol,
+      VectorSource source, int sourceIndex) {
+    super(outputCol, source, sourceIndex);
+    this.projectedName = outputCol.name();
+    this.schema = outputCol.schema();
+  }
+
   @Override
   public String name() { return projectedName; }
 
@@ -50,9 +56,6 @@ public class ResolvedTableColumn extends ResolvedColumn {
   public MaterializedField schema() { return schema; }
 
   @Override
-  public int nodeType() { return ID; }
-
-  @Override
   public String toString() {
     StringBuilder buf = new StringBuilder();
     buf
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ResolvedTuple.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ResolvedTuple.java
index c6e1510..531757a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ResolvedTuple.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ResolvedTuple.java
@@ -184,7 +184,8 @@ public abstract class ResolvedTuple implements VectorSource {
     protected AbstractMapVector outputMap;
 
     public ResolvedMap(ResolvedMapColumn parentColumn) {
-      super(parentColumn.parent().nullBuilder == null ? null : parentColumn.parent().nullBuilder.newChild());
+      super(parentColumn.parent().nullBuilder == null
+          ? null : parentColumn.parent().nullBuilder.newChild(parentColumn.name()));
       this.parentColumn = parentColumn;
     }
 
@@ -326,7 +327,7 @@ public abstract class ResolvedTuple implements VectorSource {
       return false;
     }
     for (int i = 0; i < members.size(); i++) {
-      if (members.get(i).nodeType() == ResolvedNullColumn.ID) {
+      if (members.get(i) instanceof ResolvedNullColumn) {
         return false;
       }
     }
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 f90f722..aed590f 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
@@ -21,16 +21,33 @@ import java.util.ArrayList;
 import java.util.List;
 
 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.UnresolvedSchemaColumn;
+import org.apache.drill.exec.physical.impl.scan.project.AbstractUnresolvedColumn.UnresolvedWildcardColumn;
+import org.apache.drill.exec.physical.rowSet.project.RequestedColumnImpl;
 import org.apache.drill.exec.physical.rowSet.project.RequestedTuple;
 import org.apache.drill.exec.physical.rowSet.project.RequestedTuple.RequestedColumn;
 import org.apache.drill.exec.physical.rowSet.project.RequestedTupleImpl;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.ProjectionType;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 
 /**
  * Parses and analyzes the projection list passed to the scanner. The
- * projection list is per scan, independent of any tables that the
+ * scanner accepts a projection list and a plugin-specific set of items
+ * to read. The scan operator produces a series of output batches, which
+ * (in the best case) all have the same schema. Since Drill is "schema
+ * on read", in practice batch schema may evolve. The framework tries
+ * to "smooth" such changes where possible. An output schema adds another
+ * level of stability by specifying the set of columns to project (for
+ * wildcard queries) and the types of those columns (for all queries.)
+ * <p>
+ * The projection list is per scan, independent of any tables that the
  * scanner might scan. The projection list is then used as input to the
  * per-table projection planning.
- * <p>
+ *
+ * <h4>Overview</h4>
+ *
  * In most query engines, this kind of projection analysis is done at
  * plan time. But, since Drill is schema-on-read, we don't know the
  * available columns, or their types, until we start scanning a table.
@@ -46,9 +63,8 @@ import org.apache.drill.exec.physical.rowSet.project.RequestedTupleImpl;
  * table and scan-level projections.
  * </ul>
  * <p>
- * Accepts the inputs needed to
- * plan a projection, builds the mappings, and constructs the projection
- * mapping object.
+ * Accepts the inputs needed to plan a projection, builds the mappings,
+ * and constructs the projection mapping object.
  * <p>
  * Builds the per-scan projection plan given a set of projected columns.
  * Determines the output schema, which columns to project from the data
@@ -58,9 +74,11 @@ import org.apache.drill.exec.physical.rowSet.project.RequestedTupleImpl;
  * columns to appear in the output) is specified after the SELECT keyword.
  * In Relational theory, projection is about columns, selection is about
  * rows...
- * <p>
+ *
+ * <h4>Projection Mappings</h4>
+ *
  * Mappings can be based on three primary use cases:
- * <ul>
+ * <p><ul>
  * <li><tt>SELECT *</tt>: Project all data source columns, whatever they happen
  * to be. Create columns using names from the data source. The data source
  * also determines the order of columns within the row.</li>
@@ -79,7 +97,7 @@ import org.apache.drill.exec.physical.rowSet.project.RequestedTupleImpl;
  * </ul>
  * Names in the SELECT list can reference any of five distinct types of output
  * columns:
- * <ul>
+ * <p><ul>
  * <li>Wildcard ("*") column: indicates the place in the projection list to insert
  * the table columns once found in the table projection plan.</li>
  * <li>Data source columns: columns from the underlying table. The table
@@ -94,6 +112,25 @@ import org.apache.drill.exec.physical.rowSet.project.RequestedTupleImpl;
  * parts of the path name of the file.</li>
  * </ul>
  *
+ * <h4>Projection with a Schema</h4>
+ *
+ * The client can provide an <i>output schema</i> that defines the types (and
+ * defaults) for the tuple produced by the scan. When a schema is provided,
+ * the above use cases are extended as follows:
+ * <p><ul>
+ * <li><tt>SELECT *</tt> with strict schema: All columns in the output schema
+ * are projected, and only those columns. If a reader offers additional columns,
+ * those columns are ignored. If the reader omits output columns, the default value
+ * (if any) for the column is used.</li>
+ * <li><tt>SELECT *</tt> with a non-strict schema: the output tuple contains all
+ * columns from the output schema as explained above. In addition, if the reader
+ * provides any columns not in the output schema, those columns are appended to
+ * the end of the tuple. (That is, the output schema acts as it it were from
+ * an imaginary "0th" reader.)</li>
+ * <li>Explicit projection: only the requested columns appear, whether from the
+ * output schema, the reader, or  as nulls.</li>
+ * </ul>
+ * <p>
  * @see {@link ImplicitColumnExplorer}, the class from which this class
  * evolved
  */
@@ -101,6 +138,77 @@ import org.apache.drill.exec.physical.rowSet.project.RequestedTupleImpl;
 public class ScanLevelProjection {
 
   /**
+   * Identifies the kind of projection done for this scan.
+   */
+  public enum ScanProjectionType {
+
+    /**
+     * No projection. Occurs for SELECT COUNT(*) ... queries.
+     */
+    EMPTY,
+
+    /**
+     * Wildcard. Occurs for SELECT * ... queries when no output schema is
+     * available. The scan projects all columns from all readers, using the
+     * type from that reader. Schema "smoothing", if enabled, will attempt
+     * to preserve column order, type and mode from one reader to the next.
+     */
+    WILDCARD,
+
+    /**
+     * Explicit projection. Occurs for SELECT a, b, c ... queries, whether or
+     * not an output schema is present. In this case, the projection list
+     * identifies the set of columns to project and their order. The output
+     * schema, if present, specifies data types and modes.
+     */
+    EXPLICIT,
+
+    /**
+     * Wildcard query expanded using an output schema. Occurs for a
+     * SELECT * ... query with an output schema. The set of projected columns
+     * are those from the output schema, in the order specified by the schema,
+     * with names (and name case) specified by the schema. In this mode, the
+     * schema is partial: readers may include additional columns which are
+     * appended to those provided by the schema.
+     * <p>
+     * TODO: Provide a strict mode that forces the use of the types and modes
+     * from the output schema. In lenient mode, the framework will adjust
+     * mode to allow the query to succeed (changing a required mode to
+     * optional, say, if the column is not provided by the reader and has
+     * no default. Strict mode would fail the query in this case.)
+     * <p>
+     * TODO: Enable schema smoothing in this case: use that mechanism to
+     * smooth over the "extra" reader columns.
+     */
+    SCHEMA_WILDCARD,
+
+    /**
+     * Wldcard query expanded using an output schema in "strict" mode.
+     * Only columns from the output schema will be projected. Unlike the
+     * {@link SCHEMA_WILDCARD} mode, if a reader offers columns not in the
+     * output schema, they will be ignored. That is, a SELECT * query expands
+     * to exactly the columns in the schema.
+     * <p>
+     * TODO: Provide a strict column mode that will fail the query if a projected
+     * column is required, has no default, and is not provided by the reader. In
+     * the normal lenient mode, the scan framework will adjust the data mode to
+     * optional so that the query will run.
+     */
+    STRICT_SCHEMA_WILDCARD;
+
+    public boolean isWildcard() {
+      return this == WILDCARD ||
+             this == SCHEMA_WILDCARD ||
+             this == STRICT_SCHEMA_WILDCARD;
+    }
+
+    public boolean hasSchema() {
+      return this == SCHEMA_WILDCARD ||
+             this == STRICT_SCHEMA_WILDCARD;
+    }
+  }
+
+  /**
    * Interface for add-on parsers, avoids the need to create
    * a single, tightly-coupled parser for all types of columns.
    * The main parser handles wildcards and assumes the rest of
@@ -119,6 +227,7 @@ public class ScanLevelProjection {
   // Input
 
   protected final List<SchemaPath> projectionList;
+  protected final TupleMetadata outputSchema;
 
   // Configuration
 
@@ -146,8 +255,7 @@ public class ScanLevelProjection {
    */
 
   protected RequestedTuple readerProjection;
-  protected boolean hasWildcard;
-  protected boolean emptyProjection = true;
+  protected ScanProjectionType projectionType = ScanProjectionType.EMPTY;
 
   /**
    * Specify the set of columns in the SELECT list. Since the column list
@@ -159,8 +267,16 @@ public class ScanLevelProjection {
    */
   public ScanLevelProjection(List<SchemaPath> projectionList,
       List<ScanProjectionParser> parsers) {
+    this(projectionList, parsers, null);
+  }
+
+  public ScanLevelProjection(List<SchemaPath> projectionList,
+      List<ScanProjectionParser> parsers,
+      TupleMetadata outputSchema) {
     this.projectionList = projectionList;
     this.parsers = parsers;
+    this.outputSchema = outputSchema == null || outputSchema.size() == 0
+        ? null : outputSchema;
     doParse();
   }
 
@@ -180,7 +296,7 @@ public class ScanLevelProjection {
       }
     }
 
-    // Second pass: process remaining columns.
+    // Second pass: process projected columns.
 
     for (RequestedColumn inCol : outputProjection.projections()) {
       if (inCol.isWildcard()) {
@@ -194,18 +310,27 @@ public class ScanLevelProjection {
       parser.build();
     }
 
+    buildReaderProjection();
+  }
+
+  private void buildReaderProjection() {
+
     // Create the reader projection which includes either all columns
     // (saw a wildcard) or just the unresolved columns (which excludes
     // implicit columns.)
+    //
+    // Note that only the wildcard without schema can omit the output
+    // projection. With a schema, we want the schema columns (which may
+    // or may not correspond to reader columns.)
 
     List<RequestedColumn> outputProj;
-    if (hasWildcard()) {
+    if (projectionType == ScanProjectionType.WILDCARD) {
       outputProj = null;
     } else {
       outputProj = new ArrayList<>();
       for (ColumnProjection col : outputCols) {
-        if (col instanceof UnresolvedColumn) {
-          outputProj.add(((UnresolvedColumn) col).element());
+        if (col instanceof AbstractUnresolvedColumn) {
+          outputProj.add(((AbstractUnresolvedColumn) col).element());
         }
       }
     }
@@ -227,6 +352,10 @@ public class ScanLevelProjection {
       throw new IllegalArgumentException("Duplicate * entry in project list");
     }
 
+    // Expand schema columns, if provided
+
+    expandOutputSchema();
+
     // Remember the wildcard position, if we need to insert it.
     // Ensures that the main wildcard expansion occurs before add-on
     // columns.
@@ -249,11 +378,32 @@ public class ScanLevelProjection {
     // If not consumed, put the wildcard column into the projection list as a
     // placeholder to be filled in later with actual table columns.
 
-    if (wildcardPosn != -1) {
-      UnresolvedColumn wildcardCol = new UnresolvedColumn(inCol, UnresolvedColumn.WILDCARD);
-      outputCols.add(wildcardPosn, wildcardCol);
-      hasWildcard = true;
-      emptyProjection = false;
+    if (hasOutputSchema()) {
+      projectionType =
+          outputSchema.getBooleanProperty(TupleMetadata.IS_STRICT_SCHEMA_PROP)
+          ? ScanProjectionType.STRICT_SCHEMA_WILDCARD
+          : ScanProjectionType.SCHEMA_WILDCARD;
+    } else if (wildcardPosn != -1) {
+      outputCols.add(wildcardPosn, new UnresolvedWildcardColumn(inCol));
+      projectionType = ScanProjectionType.WILDCARD;
+    }
+  }
+
+  private void expandOutputSchema() {
+    if (outputSchema == null) {
+      return;
+    }
+
+    // Expand the wildcard. From the perspective of the reader, this is an explicit
+    // projection, so enumerate the columns as though they were in the project list.
+    // 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);
+      RequestedColumn projCol = new RequestedColumnImpl(readerProjection, col.name(),
+          ProjectionType.typeFor(col.majorType()));
+      outputCols.add(new UnresolvedSchemaColumn(projCol, col));
     }
   }
 
@@ -298,13 +448,12 @@ public class ScanLevelProjection {
 
     // This is a desired table column.
 
-    addTableColumn(
-        new UnresolvedColumn(inCol, UnresolvedColumn.UNRESOLVED));
+    addTableColumn(new UnresolvedColumn(inCol));
   }
 
   public void addTableColumn(ColumnProjection outCol) {
     outputCols.add(outCol);
-    emptyProjection = false;
+    projectionType = ScanProjectionType.EXPLICIT;
   }
 
   public void addMetadataColumn(ColumnProjection outCol) {
@@ -351,14 +500,14 @@ public class ScanLevelProjection {
 
   public List<ColumnProjection> columns() { return outputCols; }
 
-  public boolean hasWildcard() { return hasWildcard; }
+  public ScanProjectionType projectionType() { return projectionType; }
 
   /**
    * Return whether this is a SELECT * query
    * @return true if this is a SELECT * query
    */
 
-  public boolean projectAll() { return hasWildcard; }
+  public boolean projectAll() { return projectionType.isWildcard(); }
 
   /**
    * Returns true if the projection list is empty. This usually
@@ -371,12 +520,16 @@ public class ScanLevelProjection {
    * the wildcard)
    */
 
-  public boolean projectNone() { return emptyProjection; }
+  public boolean projectNone() { return projectionType == ScanProjectionType.EMPTY; }
 
   public RequestedTuple rootProjection() { return outputProjection; }
 
   public RequestedTuple readerProjection() { return readerProjection; }
 
+  public boolean hasOutputSchema() { return outputSchema != null; }
+
+  public TupleMetadata outputSchema() { return outputSchema; }
+
   @Override
   public String toString() {
     return new StringBuilder()
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ScanSchemaOrchestrator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ScanSchemaOrchestrator.java
index 9269365..a1bebb8 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
@@ -24,9 +24,12 @@ import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.physical.impl.scan.project.ScanLevelProjection.ScanProjectionParser;
-import org.apache.drill.exec.physical.impl.scan.project.SchemaLevelProjection.SchemaProjectionResolver;
+import org.apache.drill.exec.physical.impl.scan.project.ReaderLevelProjection.ReaderProjectionResolver;
 import org.apache.drill.exec.physical.rowSet.impl.ResultVectorCacheImpl;
+import org.apache.drill.exec.physical.rowSet.impl.SchemaTransformer;
+import org.apache.drill.exec.physical.rowSet.impl.SchemaTransformerImpl;
 import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.vector.ValueVector;
 
 /**
@@ -149,36 +152,158 @@ public class ScanSchemaOrchestrator {
   public static final int DEFAULT_BATCH_BYTE_COUNT = ValueVector.MAX_BUFFER_SIZE;
   public static final int MAX_BATCH_ROW_COUNT = ValueVector.MAX_ROW_COUNT;
 
-  // Configuration
+  public static class ScanOrchestratorBuilder {
+
+    private MajorType nullType;
+    private MetadataManager metadataManager;
+    private int scanBatchRecordLimit = DEFAULT_BATCH_ROW_COUNT;
+    private int scanBatchByteLimit = DEFAULT_BATCH_BYTE_COUNT;
+    private List<ScanProjectionParser> parsers = new ArrayList<>();
+    private List<ReaderProjectionResolver> schemaResolvers = new ArrayList<>();
+    private boolean useSchemaSmoothing;
+    private boolean allowRequiredNullColumns;
+    private List<SchemaPath> projection;
+    private TupleMetadata outputSchema;
+    private SchemaTransformer schemaTransformer;
+
+    /**
+     * Specify an optional metadata manager. Metadata is a set of constant
+     * columns with per-reader values. For file-based sources, this is usually
+     * the implicit and partition columns; but it could be other items for other
+     * data sources.
+     *
+     * @param metadataMgr the application-specific metadata manager to use
+     * for this scan
+     */
+
+    public void withMetadata(MetadataManager metadataMgr) {
+      metadataManager = metadataMgr;
+      schemaResolvers.add(metadataManager.resolver());
+    }
 
-  /**
-   * Custom null type, if provided by the operator. If
-   * not set, the null type is the Drill default.
-   */
+    /**
+     * Specify a custom batch record count. This is the maximum number of records
+     * per batch for this scan. Readers can adjust this, but the adjustment is capped
+     * at the value specified here
+     *
+     * @param scanBatchSize maximum records per batch
+     */
+
+    public void setBatchRecordLimit(int batchRecordLimit) {
+      scanBatchRecordLimit = Math.max(1,
+          Math.min(batchRecordLimit, ValueVector.MAX_ROW_COUNT));
+    }
 
-  MajorType nullType;
+    public void setBatchByteLimit(int byteLimit) {
+      scanBatchByteLimit = Math.max(MIN_BATCH_BYTE_SIZE,
+          Math.min(byteLimit, MAX_BATCH_BYTE_SIZE));
+    }
 
-  /**
-   * Creates the metadata (file and directory) columns, if needed.
-   */
+    /**
+     * Specify the type to use for null columns in place of the standard
+     * nullable int. This type is used for all missing columns. (Readers
+     * that need per-column control need a different mechanism.)
+     *
+     * @param nullType
+     */
+
+    public void setNullType(MajorType nullType) {
+      this.nullType = nullType;
+    }
+
+    /**
+     * Enable schema smoothing: introduces an addition level of schema
+     * resolution each time a schema changes from a reader.
+     *
+     * @param flag true to enable schema smoothing, false to disable
+     */
+
+    public void enableSchemaSmoothing(boolean flag) {
+      useSchemaSmoothing = flag;
+   }
+
+    public void allowRequiredNullColumns(boolean flag) {
+      allowRequiredNullColumns = flag;
+    }
+
+    public void addParser(ScanProjectionParser parser) {
+      parsers.add(parser);
+    }
+
+    public void addResolver(ReaderProjectionResolver resolver) {
+      schemaResolvers.add(resolver);
+    }
+
+    public void setProjection(List<SchemaPath> projection) {
+      this.projection = projection;
+    }
+
+    public void setOutputSchema(TupleMetadata schema) {
+      outputSchema = schema;
+    }
+
+    public void setSchemaTransformer(SchemaTransformer transformer) {
+      this.schemaTransformer = transformer;
+    }
+  }
+
+  public static class ScanSchemaOptions {
+
+    /**
+     * 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;
+    public final List<ScanProjectionParser> parsers;
+
+    /**
+     * List of resolvers used to resolve projection columns for each
+     * new schema. Allows operators to introduce custom functionality
+     * 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 SchemaTransformer schemaTransformer;
+
+    protected ScanSchemaOptions(ScanOrchestratorBuilder builder) {
+      nullType = builder.nullType;
+      scanBatchRecordLimit = builder.scanBatchRecordLimit;
+      scanBatchByteLimit = builder.scanBatchByteLimit;
+      parsers = builder.parsers;
+      schemaResolvers = builder.schemaResolvers;
+      projection = builder.projection;
+      useSchemaSmoothing = builder.useSchemaSmoothing;
+      allowRequiredNullColumns = builder.allowRequiredNullColumns;
+      if (builder.schemaTransformer != null) {
+        // Use client-provided conversions
+        schemaTransformer = builder.schemaTransformer;
+      } else if (builder.outputSchema != null) {
+        // Use only implicit conversions
+        schemaTransformer = new SchemaTransformerImpl(builder.outputSchema);
+      } else {
+        schemaTransformer = null;
+      }
+    }
+  }
+
+  // Configuration
 
-  MetadataManager metadataManager;
-  final BufferAllocator allocator;
-  int scanBatchRecordLimit = DEFAULT_BATCH_ROW_COUNT;
-  int scanBatchByteLimit = DEFAULT_BATCH_BYTE_COUNT;
-  private final List<ScanProjectionParser> parsers = new ArrayList<>();
+  protected final BufferAllocator allocator;
+  protected final ScanSchemaOptions options;
 
   /**
-   * List of resolvers used to resolve projection columns for each
-   * new schema. Allows operators to introduce custom functionality
-   * as a plug-in rather than by copying code or subclassing this
-   * mechanism.
+   * Creates the metadata (file and directory) columns, if needed.
    */
 
-  List<SchemaProjectionResolver> schemaResolvers = new ArrayList<>();
-
-  private boolean useSchemaSmoothing;
-  boolean allowRequiredNullColumns;
+  public final MetadataManager metadataManager;
 
   // Internal state
 
@@ -200,78 +325,19 @@ public class ScanSchemaOrchestrator {
 
   VectorContainer outputContainer;
 
-  public ScanSchemaOrchestrator(BufferAllocator allocator) {
+  public ScanSchemaOrchestrator(BufferAllocator allocator, ScanOrchestratorBuilder builder) {
     this.allocator = allocator;
-  }
-
-  /**
-   * Specify an optional metadata manager. Metadata is a set of constant
-   * columns with per-reader values. For file-based sources, this is usually
-   * the implicit and partition columns; but it could be other items for other
-   * data sources.
-   *
-   * @param metadataMgr the application-specific metadata manager to use
-   * for this scan
-   */
-
-  public void withMetadata(MetadataManager metadataMgr) {
-    metadataManager = metadataMgr;
-    schemaResolvers.add(metadataManager.resolver());
-  }
-
-  /**
-   * Specify a custom batch record count. This is the maximum number of records
-   * per batch for this scan. Readers can adjust this, but the adjustment is capped
-   * at the value specified here
-   *
-   * @param scanBatchSize maximum records per batch
-   */
-
-  public void setBatchRecordLimit(int batchRecordLimit) {
-    scanBatchRecordLimit = Math.max(1,
-        Math.min(batchRecordLimit, ValueVector.MAX_ROW_COUNT));
-  }
+    this.options = new ScanSchemaOptions(builder);
 
-  public void setBatchByteLimit(int byteLimit) {
-    scanBatchByteLimit = Math.max(MIN_BATCH_BYTE_SIZE,
-        Math.min(byteLimit, MAX_BATCH_BYTE_SIZE));
-  }
-
-  /**
-   * Specify the type to use for null columns in place of the standard
-   * nullable int. This type is used for all missing columns. (Readers
-   * that need per-column control need a different mechanism.)
-   *
-   * @param nullType
-   */
-
-  public void setNullType(MajorType nullType) {
-    this.nullType = nullType;
-  }
-
-  /**
-   * Enable schema smoothing: introduces an addition level of schema
-   * resolution each time a schema changes from a reader.
-   *
-   * @param flag true to enable schema smoothing, false to disable
-   */
-
-  public void enableSchemaSmoothing(boolean flag) {
-    useSchemaSmoothing = flag;
-  }
-
-  public void allowRequiredNullColumns(boolean flag) {
-    allowRequiredNullColumns = flag;
-  }
-
-  public void build(List<SchemaPath> projection) {
-    vectorCache = new ResultVectorCacheImpl(allocator, useSchemaSmoothing);
+    vectorCache = new ResultVectorCacheImpl(allocator, options.useSchemaSmoothing);
 
     // If no metadata manager was provided, create a mock
     // version just to keep code simple.
 
-    if (metadataManager == null) {
+    if (builder.metadataManager == null) {
       metadataManager = new NoOpMetadataManager();
+    } else {
+      metadataManager = builder.metadataManager;
     }
     metadataManager.bind(vectorCache);
 
@@ -284,14 +350,18 @@ public class ScanSchemaOrchestrator {
     if (parser != null) {
       // Insert in first position so that it is ensured to see
       // any wildcard that exists
-      parsers.add(0, parser);
+      options.parsers.add(0, parser);
     }
 
     // Parse the projection list.
 
-    scanProj = new ScanLevelProjection(projection, parsers);
-    if (scanProj.hasWildcard() && useSchemaSmoothing) {
-      schemaSmoother = new SchemaSmoother(scanProj, schemaResolvers);
+    TupleMetadata outputSchema = null;
+    if (options.schemaTransformer != null) {
+      outputSchema = options.schemaTransformer.outputSchema();
+    }
+    scanProj = new ScanLevelProjection(options.projection, options.parsers, outputSchema);
+    if (scanProj.projectAll() && options.useSchemaSmoothing) {
+      schemaSmoother = new SchemaSmoother(scanProj, options.schemaResolvers);
     }
 
     // Build the output container.
@@ -299,14 +369,6 @@ public class ScanSchemaOrchestrator {
     outputContainer = new VectorContainer(allocator);
   }
 
-  public void addParser(ScanProjectionParser parser) {
-    parsers.add(parser);
-  }
-
-  public void addResolver(SchemaProjectionResolver resolver) {
-    schemaResolvers.add(resolver);
-  }
-
   public ReaderSchemaOrchestrator startReader() {
     closeReader();
     currentReader = new ReaderSchemaOrchestrator(this);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/SchemaSmoother.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/SchemaSmoother.java
index b15fe9a..6582f88 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/SchemaSmoother.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/SchemaSmoother.java
@@ -19,7 +19,7 @@ package org.apache.drill.exec.physical.impl.scan.project;
 
 import java.util.List;
 
-import org.apache.drill.exec.physical.impl.scan.project.SchemaLevelProjection.SchemaProjectionResolver;
+import org.apache.drill.exec.physical.impl.scan.project.ReaderLevelProjection.ReaderProjectionResolver;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
 
 /**
@@ -76,17 +76,17 @@ public class SchemaSmoother {
   public static class IncompatibleSchemaException extends Exception { }
 
   private final ScanLevelProjection scanProj;
-  private final List<SchemaProjectionResolver> resolvers;
+  private final List<ReaderProjectionResolver> resolvers;
   private ResolvedTuple priorSchema;
   private int schemaVersion = 0;
 
   public SchemaSmoother(ScanLevelProjection scanProj,
-      List<SchemaProjectionResolver> resolvers) {
+      List<ReaderProjectionResolver> resolvers) {
     this.scanProj = scanProj;
     this.resolvers = resolvers;
   }
 
-  public SchemaLevelProjection resolve(
+  public ReaderLevelProjection resolve(
       TupleMetadata tableSchema,
       ResolvedTuple outputTuple) {
 
@@ -111,7 +111,7 @@ public class SchemaSmoother {
     // will occur because either a type has changed or a new column has appeared.
     // (Or, this is the first schema.)
 
-    SchemaLevelProjection schemaProj = new WildcardSchemaProjection(scanProj,
+    ReaderLevelProjection schemaProj = new WildcardProjection(scanProj,
         tableSchema, outputTuple, resolvers);
     priorSchema = outputTuple;
     schemaVersion++;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/SmoothingProjection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/SmoothingProjection.java
index da199b8..fa7a7f4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/SmoothingProjection.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/SmoothingProjection.java
@@ -66,7 +66,7 @@ import org.apache.drill.exec.record.metadata.TupleMetadata;
  * any per-file variation of schema to match the up-front schema.
  */
 
-public class SmoothingProjection extends SchemaLevelProjection {
+public class SmoothingProjection extends ReaderLevelProjection {
 
   protected final List<MaterializedField> rewrittenFields = new ArrayList<>();
 
@@ -74,20 +74,18 @@ public class SmoothingProjection extends SchemaLevelProjection {
       TupleMetadata tableSchema,
       ResolvedTuple priorSchema,
       ResolvedTuple outputTuple,
-      List<SchemaProjectionResolver> resolvers) throws IncompatibleSchemaException {
+      List<ReaderProjectionResolver> resolvers) throws IncompatibleSchemaException {
 
     super(resolvers);
 
     for (ResolvedColumn priorCol : priorSchema.columns()) {
-      switch (priorCol.nodeType()) {
-        case ResolvedTableColumn.ID:
-        case ResolvedNullColumn.ID:
-          // This is a regular column known to this base framework.
-          resolveColumn(outputTuple, priorCol, tableSchema);
-          break;
-        default:
-          // The column is one known to an add-on mechanism.
-          resolveSpecial(outputTuple, priorCol, tableSchema);
+      if (priorCol instanceof ResolvedTableColumn ||
+        priorCol instanceof  ResolvedNullColumn) {
+        // This is a regular column known to this base framework.
+        resolveColumn(outputTuple, priorCol, tableSchema);
+      } else {
+        // The column is one known to an add-on mechanism.
+        resolveSpecial(outputTuple, priorCol, tableSchema);
       }
     }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/WildcardSchemaProjection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/WildcardProjection.java
similarity index 86%
copy from exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/WildcardSchemaProjection.java
copy to exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/WildcardProjection.java
index d1cfbe1..d2b5def 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/WildcardSchemaProjection.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/WildcardProjection.java
@@ -19,6 +19,7 @@ package org.apache.drill.exec.physical.impl.scan.project;
 
 import java.util.List;
 
+import org.apache.drill.exec.physical.impl.scan.project.AbstractUnresolvedColumn.UnresolvedWildcardColumn;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
 
@@ -29,15 +30,15 @@ import org.apache.drill.exec.record.metadata.TupleMetadata;
  * to create null columns. Example: SELECT *
  */
 
-public class WildcardSchemaProjection extends SchemaLevelProjection {
+public class WildcardProjection extends ReaderLevelProjection {
 
-  public WildcardSchemaProjection(ScanLevelProjection scanProj,
+  public WildcardProjection(ScanLevelProjection scanProj,
       TupleMetadata tableSchema,
       ResolvedTuple rootTuple,
-      List<SchemaProjectionResolver> resolvers) {
+      List<ReaderProjectionResolver> resolvers) {
     super(resolvers);
     for (ColumnProjection col : scanProj.columns()) {
-      if (col.nodeType() == UnresolvedColumn.WILDCARD) {
+      if (col instanceof UnresolvedWildcardColumn) {
         projectAllColumns(rootTuple, tableSchema);
       } else {
         resolveSpecial(rootTuple, col, tableSchema);
@@ -49,8 +50,6 @@ public class WildcardSchemaProjection extends SchemaLevelProjection {
    * Project all columns from table schema to the output, in table
    * schema order. Since we accept any map columns as-is, no need
    * to do recursive projection.
-   *
-   * @param tableSchema
    */
 
   private void projectAllColumns(ResolvedTuple rootTuple, TupleMetadata tableSchema) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/WildcardSchemaProjection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/WildcardSchemaProjection.java
index d1cfbe1..ea5c750 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/WildcardSchemaProjection.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/WildcardSchemaProjection.java
@@ -19,46 +19,75 @@ package org.apache.drill.exec.physical.impl.scan.project;
 
 import java.util.List;
 
-import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.physical.impl.scan.project.AbstractUnresolvedColumn.UnresolvedSchemaColumn;
+import org.apache.drill.exec.physical.impl.scan.project.ScanLevelProjection.ScanProjectionType;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
 
 /**
- * Perform a wildcard projection. In this case, the query wants all
- * columns in the source table, so the table drives the final projection.
- * Since we include only those columns in the table, there is no need
- * to create null columns. Example: SELECT *
+ * Perform a wildcard projection with an associated output schema.
+ * Matches the reader schema against the output schema. If a column
+ * appears, it is projected into the output schema. If not found,
+ * then a null column (as defined by the output schema) is projected.
+ * <p>
+ * If the schema is strict, then we stop here. If not strict, then
+ * any unmatched reader schema columns are appended to the output
+ * tuple.
  */
 
-public class WildcardSchemaProjection extends SchemaLevelProjection {
+public class WildcardSchemaProjection extends ReaderLevelProjection {
 
   public WildcardSchemaProjection(ScanLevelProjection scanProj,
-      TupleMetadata tableSchema,
+      TupleMetadata readerSchema,
       ResolvedTuple rootTuple,
-      List<SchemaProjectionResolver> resolvers) {
+      List<ReaderProjectionResolver> resolvers) {
     super(resolvers);
+
+    // Match each column expanded from the output schema against the
+    // columns provided by the reader.
+
+    boolean readerProjectionMap[] = new boolean[readerSchema.size()];
     for (ColumnProjection col : scanProj.columns()) {
-      if (col.nodeType() == UnresolvedColumn.WILDCARD) {
-        projectAllColumns(rootTuple, tableSchema);
+      if (col instanceof UnresolvedSchemaColumn) {
+
+        // Look for a match in the reader schema
+
+        ColumnMetadata readerCol = readerSchema.metadata(col.name());
+        UnresolvedSchemaColumn schemaCol = (UnresolvedSchemaColumn) col;
+        if (readerCol == null) {
+
+          // No match, project a null column
+
+          rootTuple.add(rootTuple.nullBuilder.add(schemaCol.metadata()));
+        } else {
+
+          // Is a match, project this reader column
+
+          int index = readerSchema.index(col.name());
+          readerProjectionMap[index] = true;
+          rootTuple.add(
+              new ResolvedTableColumn(schemaCol.metadata(), rootTuple, index));
+        }
       } else {
-        resolveSpecial(rootTuple, col, tableSchema);
+
+        // Not a schema column, handle specially
+
+        resolveSpecial(rootTuple, col, readerSchema);
       }
     }
-  }
 
-  /**
-   * Project all columns from table schema to the output, in table
-   * schema order. Since we accept any map columns as-is, no need
-   * to do recursive projection.
-   *
-   * @param tableSchema
-   */
-
-  private void projectAllColumns(ResolvedTuple rootTuple, TupleMetadata tableSchema) {
-    for (int i = 0; i < tableSchema.size(); i++) {
-      MaterializedField colSchema = tableSchema.column(i);
-      rootTuple.add(
-          new ResolvedTableColumn(colSchema.getName(),
-              colSchema, rootTuple, i));
+    // If lenient wildcard projection, add unmatched reader columns.
+
+    if (scanProj.projectionType() == ScanProjectionType.SCHEMA_WILDCARD) {
+      for (int i = 0; i < readerProjectionMap.length; i++) {
+        if (readerProjectionMap[i]) {
+          continue;
+        }
+        ColumnMetadata readerCol = readerSchema.metadata(i);
+        rootTuple.add(
+            new ResolvedTableColumn(readerCol.name(),
+                readerCol.schema(), rootTuple, i));
+      }
     }
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortRecordBatchBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortRecordBatchBuilder.java
index 1eb7078..ce8a3aa 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortRecordBatchBuilder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortRecordBatchBuilder.java
@@ -98,7 +98,6 @@ public class SortRecordBatchBuilder implements AutoCloseable {
     return true;
   }
 
-  @SuppressWarnings("resource")
   public void add(RecordBatchData rbd) {
     long batchBytes = getSize(rbd.getContainer());
     if (batchBytes == 0 && batches.size() > 0) {
@@ -135,7 +134,6 @@ public class SortRecordBatchBuilder implements AutoCloseable {
     return batches.isEmpty();
   }
 
-  @SuppressWarnings("resource")
   public void build(VectorContainer outputContainer) throws SchemaChangeException {
     outputContainer.clear();
     if (batches.keySet().size() > 1) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/spill/SpillSet.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/spill/SpillSet.java
index 7d266de..3a2e668 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/spill/SpillSet.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/spill/SpillSet.java
@@ -305,14 +305,12 @@ public class SpillSet {
       dir.deleteOnExit();
     }
 
-    @SuppressWarnings("resource")
     @Override
     public WritableByteChannel createForWrite(String fileName) throws IOException {
       return FileChannel.open(new File(baseDir, fileName).toPath(), StandardOpenOption.CREATE, StandardOpenOption.WRITE);
     }
 
-    @SuppressWarnings("resource")
-    @Override
+     @Override
     public InputStream openForInput(String fileName) throws IOException {
       return new CountingInputStream(
                 new BufferedInputStream(
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/GenericSV4Copier.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/GenericSV4Copier.java
index c676841..e81eb43 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/GenericSV4Copier.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/GenericSV4Copier.java
@@ -28,7 +28,6 @@ public class GenericSV4Copier extends AbstractSV4Copier {
   public GenericSV4Copier(RecordBatch incomingBatch, VectorContainer outputContainer,
                           SchemaChangeCallBack callBack) {
     for(VectorWrapper<?> vv : incomingBatch){
-      @SuppressWarnings("resource")
       ValueVector v = vv.getValueVectors()[0];
       v.makeTransferPair(outputContainer.addOrGet(v.getField(), callBack));
     }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java
index e83fddf..90656ae 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java
@@ -88,6 +88,7 @@ public class UnionAllRecordBatch extends AbstractBinaryRecordBatch<UnionAll> {
     right.kill(sendUpstream);
   }
 
+  @Override
   protected void buildSchema() throws SchemaChangeException {
     if (! prefetchFirstBatchFromBothSides()) {
       state = BatchState.DONE;
@@ -153,7 +154,6 @@ public class UnionAllRecordBatch extends AbstractBinaryRecordBatch<UnionAll> {
   }
 
 
-  @SuppressWarnings("resource")
   private IterOutcome doWork(BatchStatusWrappper batchStatus, boolean newSchema) throws ClassTransformationException, IOException, SchemaChangeException {
     Preconditions.checkArgument(batchStatus.batch.getSchema().getFieldCount() == container.getSchema().getFieldCount(),
         "Input batch and output batch have different field counthas!");
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unnest/UnnestRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unnest/UnnestRecordBatch.java
index 1c8336d..603c583 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unnest/UnnestRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unnest/UnnestRecordBatch.java
@@ -145,6 +145,7 @@ public class UnnestRecordBatch extends AbstractTableFunctionRecordBatch<UnnestPO
     return recordCount;
   }
 
+  @Override
   protected void killIncoming(boolean sendUpstream) {
     //
     // In some cases we need to return a predetermined state from a call to next. These are:
@@ -246,7 +247,6 @@ public class UnnestRecordBatch extends AbstractTableFunctionRecordBatch<UnnestPO
     return this.container;
   }
 
-  @SuppressWarnings("resource")
   private void setUnnestVector() {
     final MaterializedField field = incoming.getSchema().getColumn(unnestTypedFieldId.getFieldIds()[0]);
     final RepeatedValueVector vector;
@@ -268,6 +268,7 @@ public class UnnestRecordBatch extends AbstractTableFunctionRecordBatch<UnnestPO
     unnest.setUnnestField(vector);
   }
 
+  @Override
   protected IterOutcome doWork() {
     Preconditions.checkNotNull(lateral);
     unnest.setOutputCount(memoryManager.getOutputRowCount());
@@ -319,7 +320,6 @@ public class UnnestRecordBatch extends AbstractTableFunctionRecordBatch<UnnestPO
    * the end of one of the other vectors while we are copying the data of the other vectors alongside each new unnested
    * value coming out of the repeated field.)
    */
-  @SuppressWarnings("resource")
   private TransferPair getUnnestFieldTransferPair(FieldReference reference) {
     final int[] typeFieldIds = unnestTypedFieldId.getFieldIds();
     final Class<?> vectorClass = incoming.getSchema().getColumn(typeFieldIds[0]).getValueClass();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unorderedreceiver/UnorderedReceiverCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unorderedreceiver/UnorderedReceiverCreator.java
index 3dcdfc4..64b0614 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unorderedreceiver/UnorderedReceiverCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unorderedreceiver/UnorderedReceiverCreator.java
@@ -29,7 +29,6 @@ import org.apache.drill.exec.work.batch.RawBatchBuffer;
 
 public class UnorderedReceiverCreator implements BatchCreator<UnorderedReceiver> {
 
-  @SuppressWarnings("resource")
   @Override
   public UnorderedReceiverBatch getBatch(ExecutorFragmentContext context, UnorderedReceiver receiver, List<RecordBatch> children)
       throws ExecutionSetupException {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/BatchValidator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/BatchValidator.java
index 11f2efb..83923c9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/BatchValidator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/BatchValidator.java
@@ -124,14 +124,13 @@ public class BatchValidator {
     if (vector instanceof RepeatedVarCharVector) {
       dataLength = ((RepeatedVarCharVector) vector).getOffsetVector().getValueCapacity();
     } else if (vector instanceof RepeatedFixedWidthVectorLike) {
-      dataLength = ((BaseDataValueVector) ((BaseRepeatedValueVector) vector).getDataVector()).getBuffer().capacity();
+      dataLength = ((BaseDataValueVector) vector.getDataVector()).getBuffer().capacity();
     }
     int itemCount = validateOffsetVector(name + "-offsets", vector.getOffsetVector(), rowCount, dataLength);
 
     // Special handling of repeated VarChar vectors
     // The nested data vectors are not quite exactly like top-level vectors.
 
-    @SuppressWarnings("resource")
     ValueVector dataVector = vector.getDataVector();
     if (dataVector instanceof VariableWidthVector) {
       validateVariableWidthVector(name + "-data", (VariableWidthVector) dataVector, itemCount);
@@ -187,7 +186,6 @@ public class BatchValidator {
     // Validate a VarChar vector because it is common.
 
     if (vector instanceof NullableVarCharVector) {
-      @SuppressWarnings("resource")
       VarCharVector values = ((NullableVarCharVector) vector).getValuesVector();
       validateVarCharVector(name + "-values", values, rowCount);
     }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
index 262a241..c8fa175 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
@@ -240,7 +240,7 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
           mSorter.clear();
         }
         for(Iterator<Path> iter = this.currSpillDirs.iterator(); iter.hasNext(); iter.remove()) {
-            Path path = (Path)iter.next();
+            Path path = iter.next();
             try {
                 if (fs != null && path != null && fs.exists(path)) {
                     if (fs.delete(path, true)) {
@@ -266,7 +266,6 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
       case OK:
       case OK_NEW_SCHEMA:
         for (VectorWrapper<?> w : incoming) {
-          @SuppressWarnings("resource")
           ValueVector v = container.addOrGet(w.getField());
           if (v instanceof AbstractContainerVector) {
             w.getValueVector().makeTransferPair(v); // Can we remove this hack?
@@ -291,7 +290,6 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
     }
   }
 
-  @SuppressWarnings("resource")
   @Override
   public IterOutcome innerNext() {
     if (schema != null) {
@@ -553,7 +551,6 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
       if (batchGroups.size() == 0) {
         break;
       }
-      @SuppressWarnings("resource")
       BatchGroup batch = batchGroups.pollLast();
       assert batch != null : "Encountered a null batch during merge and spill operation";
       batchGroupList.add(batch);
@@ -629,7 +626,6 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
     SelectionVector2 sv2 = new SelectionVector2(oAllocator);
     if (!sv2.allocateNewSafe(incoming.getRecordCount())) {
       try {
-        @SuppressWarnings("resource")
         final BatchGroup merged = mergeAndSpill(batchGroups);
         if (merged != null) {
           spilledBatchGroups.add(merged);
@@ -800,10 +796,8 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
         copier.close();
       }
 
-      @SuppressWarnings("resource")
       BufferAllocator allocator = spilling ? copierAllocator : oAllocator;
       for (VectorWrapper<?> i : batch) {
-        @SuppressWarnings("resource")
         ValueVector v = TypeHelper.getNewVector(i.getField(), allocator);
         outputContainer.add(v);
       }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/MSortTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/MSortTemplate.java
index 9b10d43..24f066e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/MSortTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/MSortTemplate.java
@@ -73,7 +73,6 @@ public abstract class MSortTemplate implements MSorter, IndexedSortable {
         throw new UnsupportedOperationException(String.format("Missing batch. batch: %d newBatch: %d", batch, newBatch));
       }
     }
-    @SuppressWarnings("resource")
     final DrillBuf drillBuf = allocator.buffer(4 * totalCount);
 
     try {
@@ -152,7 +151,6 @@ public abstract class MSortTemplate implements MSorter, IndexedSortable {
       if (outIndex < vector4.getTotalCount()) {
         copyRun(outIndex, vector4.getTotalCount());
       }
-      @SuppressWarnings("resource")
       final SelectionVector4 tmp = aux.createNewWrapperCurrent(desiredRecordBatchCount);
       aux.clear();
       aux = vector4.createNewWrapperCurrent(desiredRecordBatchCount);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BufferedBatches.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BufferedBatches.java
index 669df17..fb7c3ae 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BufferedBatches.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BufferedBatches.java
@@ -78,7 +78,6 @@ public class BufferedBatches {
 
   public int size() { return bufferedBatches.size(); }
 
-  @SuppressWarnings("resource")
   public void add(VectorAccessible incoming, long batchSize) {
     // Convert the incoming batch to the agreed-upon schema.
     // No converted batch means we got an empty input batch.
@@ -114,7 +113,6 @@ public class BufferedBatches {
    * @return the converted batch, or null if the incoming batch is empty
    */
 
-  @SuppressWarnings("resource")
   private VectorContainer convertBatch(VectorAccessible incoming) {
 
     // Must accept the batch even if no records. Then clear
@@ -164,7 +162,6 @@ public class BufferedBatches {
     return sv2;
   }
 
-  @SuppressWarnings("resource")
   private void bufferBatch(VectorContainer convertedBatch, SelectionVector2 sv2, long netSize) {
     BufferAllocator allocator = context.getAllocator();
     RecordBatchData rbd = new RecordBatchData(convertedBatch, allocator);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/ExternalSortBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/ExternalSortBatch.java
index 8fc4a74..e6046a3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/ExternalSortBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/ExternalSortBatch.java
@@ -275,7 +275,6 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
       case OK:
       case OK_NEW_SCHEMA:
         for (VectorWrapper<?> w : incoming) {
-          @SuppressWarnings("resource")
           ValueVector v = container.addOrGet(w.getField());
           if (v instanceof AbstractContainerVector) {
             w.getValueVector().makeTransferPair(v); // Can we remove this hack?
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MSortTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MSortTemplate.java
index 8972939..cbec0ab 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MSortTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MSortTemplate.java
@@ -87,7 +87,6 @@ public abstract class MSortTemplate implements MSorter, IndexedSortable {
 
     // Create a temporary SV4 to hold the merged results.
 
-    @SuppressWarnings("resource")
     final DrillBuf drillBuf = allocator.buffer(4 * totalCount);
     desiredRecordBatchCount = Math.min(outputBatchSize, desiredBatchSize);
     desiredRecordBatchCount = Math.min(desiredRecordBatchCount, totalCount);
@@ -184,7 +183,6 @@ public abstract class MSortTemplate implements MSorter, IndexedSortable {
       if (outIndex < totalCount) {
         copyRun(outIndex, totalCount);
       }
-      @SuppressWarnings("resource")
       final SelectionVector4 tmp = aux.createNewWrapperCurrent(desiredRecordBatchCount);
       aux.clear();
       aux = vector4.createNewWrapperCurrent(desiredRecordBatchCount);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopierTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopierTemplate.java
index b48c012..46b7a49 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopierTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopierTemplate.java
@@ -48,7 +48,6 @@ public abstract class PriorityQueueCopierTemplate implements PriorityQueueCopier
     this.outgoing = outgoing;
     this.size = batchGroups.size();
 
-    @SuppressWarnings("resource")
     final DrillBuf drillBuf = allocator.buffer(4 * size);
     vector4 = new SelectionVector4(drillBuf, size, Character.MAX_VALUE);
     doSetup(hyperBatch, outgoing);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopierWrapper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopierWrapper.java
index 6a8e1b1..b9a0edb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopierWrapper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopierWrapper.java
@@ -132,7 +132,6 @@ public class PriorityQueueCopierWrapper extends BaseSortWrapper {
     // Initialize the value vectors for the output container
 
     for (VectorWrapper<?> i : batch) {
-      @SuppressWarnings("resource")
       ValueVector v = TypeHelper.getNewVector(i.getField(), context.getAllocator());
       outputContainer.add(v);
     }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/BuildFromSchema.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/BuildFromSchema.java
index 6392906..c750f8c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/BuildFromSchema.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/BuildFromSchema.java
@@ -106,7 +106,6 @@ public class BuildFromSchema {
   }
 
   private void buildColumn(ParentShim parent, ColumnMetadata colSchema) {
-
     if (colSchema.isMultiList()) {
       buildRepeatedList(parent, colSchema);
     } else if (colSchema.isMap()) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/DefaultSchemaTransformer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/DefaultSchemaTransformer.java
index 5853bf6..d1d53ab 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/DefaultSchemaTransformer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/DefaultSchemaTransformer.java
@@ -19,6 +19,7 @@ package org.apache.drill.exec.physical.rowSet.impl;
 
 import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.apache.drill.exec.record.metadata.ProjectionType;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.vector.accessor.ScalarWriter;
 import org.apache.drill.exec.vector.accessor.convert.AbstractWriteConverter;
 import org.apache.drill.exec.vector.accessor.convert.ColumnConversionFactory;
@@ -70,4 +71,7 @@ public class DefaultSchemaTransformer implements SchemaTransformer {
       ProjectionType projType) {
     return new DefaultColumnTransformer(inputSchema, projType);
   }
+
+  @Override
+  public TupleMetadata outputSchema() { return null; }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/SchemaTransformer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/SchemaTransformer.java
index 619994d..cdcf8df 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/SchemaTransformer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/SchemaTransformer.java
@@ -19,6 +19,7 @@ package org.apache.drill.exec.physical.rowSet.impl;
 
 import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.apache.drill.exec.record.metadata.ProjectionType;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.vector.accessor.convert.ColumnConversionFactory;
 
 /**
@@ -40,5 +41,6 @@ public interface SchemaTransformer {
     ColumnMetadata outputSchema();
   }
 
+  TupleMetadata outputSchema();
   ColumnTransform transform(ColumnMetadata inputSchema, ProjectionType projType);
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/SchemaTransformerImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/SchemaTransformerImpl.java
index d17ef39..8c92701 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/SchemaTransformerImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/SchemaTransformerImpl.java
@@ -185,7 +185,7 @@ public class SchemaTransformerImpl implements SchemaTransformer {
    * by this operator
    * @param defn a description of the required conversion. This method is
    * required to do nothing of conversion type is
-   * {@link ProjectionType.EXPLICIT} and the conversion class is null, meaning
+   * {@link ScanProjectionType.EXPLICIT} and the conversion class is null, meaning
    * that no standard conversion is available
    * @return a column transformer factory to implement a custom conversion,
    * or null to use the standard conversion
@@ -194,4 +194,7 @@ public class SchemaTransformerImpl implements SchemaTransformer {
       ColumnMetadata outputDefn, ConversionDefn defn) {
     return null;
   }
+
+  @Override
+  public TupleMetadata outputSchema() { return outputSchema; }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/TupleState.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/TupleState.java
index eace69e..7bc419f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/TupleState.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/TupleState.java
@@ -537,16 +537,15 @@ public abstract class TupleState extends ContainerState
         continue;
       }
 
-      // If this is a new column added since the last
-      // output, then we may have to add the column to this output.
-      // For the row itself, and for maps outside of unions, If the column was
-      // added after the output schema version cutoff, skip that column for now.
-      // But, if this tuple is within a union,
-      // then we always add all columns because union semantics are too
-      // muddy to play the deferred column game. Further, all columns in
-      // a map within a union must be nullable, so we know we can fill
-      // the column with nulls. (Something that is not true for normal
-      // maps.)
+      // If this is a new column added since the lastoutput, then we may have
+      // to add the column to this output. For the row itself, and for maps
+      // outside of unions, If the column wasadded after the output schema
+      // version cutoff, skip that column for now. But, if this tuple is
+      // within a union, then we always add all columns because union
+      // semantics are too muddy to play the deferred column game. Further,
+      // all columns in a map within a union must be nullable, so we know we
+      // can fill the column with nulls. (Something that is not true for
+      // normal maps.)
 
       if (i > prevHarvestIndex && (! isVersioned() || colState.addVersion <= curSchemaVersion)) {
         colState.buildOutput(this);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/project/RequestedColumnImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/project/RequestedColumnImpl.java
index 4e4b5c9..59b5103 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/project/RequestedColumnImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/project/RequestedColumnImpl.java
@@ -52,6 +52,12 @@ public class RequestedColumnImpl implements RequestedColumn {
     setType();
   }
 
+  public RequestedColumnImpl(RequestedTuple parent, String name, ProjectionType type) {
+    this.parent = parent;
+    this.name = name;
+    this.type = type;
+  }
+
   @Override
   public String name() { return name; }
   @Override
@@ -62,14 +68,10 @@ public class RequestedColumnImpl implements RequestedColumn {
   public boolean isSimple() { return type == ProjectionType.UNSPECIFIED; }
 
   @Override
-  public boolean isArray() {
-    return type == ProjectionType.ARRAY || type == ProjectionType.TUPLE_ARRAY;
-  }
+  public boolean isArray() { return type.isArray(); }
 
   @Override
-  public boolean isTuple() {
-    return type == ProjectionType.TUPLE || type == ProjectionType.TUPLE_ARRAY;
-  }
+  public boolean isTuple() { return type.isTuple(); }
 
   public RequestedTuple asTuple() {
     if (members == null) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/HyperVectorWrapper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/HyperVectorWrapper.java
index 16d7e3a..86fa0bf 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/HyperVectorWrapper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/HyperVectorWrapper.java
@@ -83,7 +83,6 @@ public class HyperVectorWrapper<T extends ValueVector> implements VectorWrapper<
     }
   }
 
-  @SuppressWarnings("resource")
   @Override
   public VectorWrapper<?> getChildWrapper(int[] ids) {
     if (ids.length == 1) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
index cd3a22f..224ed6f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
@@ -75,7 +75,6 @@ public class RecordBatchLoader implements VectorAccessible, Iterable<VectorWrapp
    * @throws SchemaChangeException
    *   TODO:  Clean:  DRILL-2933  load(...) never actually throws SchemaChangeException.
    */
-  @SuppressWarnings("resource")
   public boolean load(RecordBatchDef def, DrillBuf buf) throws SchemaChangeException {
     if (logger.isTraceEnabled()) {
       logger.trace("Loading record batch with def {} and data {}", def, buf);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchSizer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchSizer.java
index c89d512..679f173 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchSizer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchSizer.java
@@ -59,7 +59,7 @@ public class RecordBatchSizer {
 
   public static long multiplyByFactors(long size, double... factors)
   {
-    double doubleSize = (double) size;
+    double doubleSize = size;
 
     for (double factor: factors) {
       doubleSize *= factor;
@@ -70,7 +70,7 @@ public class RecordBatchSizer {
 
   public static long multiplyByFactor(long size, double factor)
   {
-    return (long) (((double) size) * factor);
+    return (long) ((size) * factor);
   }
 
   /**
@@ -447,7 +447,6 @@ public class RecordBatchSizer {
       }
     }
 
-    @SuppressWarnings("resource")
     private int getElementCount(ValueVector v) {
       // Repeated vectors are special: they have an associated offset vector
       // that changes the value count of the contained vectors.
@@ -829,7 +828,7 @@ public class RecordBatchSizer {
     if (denom == 0) {
       return 0;
     }
-    return (int) Math.ceil((double) num / denom);
+    return (int) Math.ceil(num / denom);
   }
 
   public int rowCount() { return rowCount; }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaUtil.java
index 9cdbdb2..0deee65 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaUtil.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaUtil.java
@@ -94,7 +94,6 @@ public class SchemaUtil {
     return s;
   }
 
-  @SuppressWarnings("resource")
   private static  ValueVector coerceVector(ValueVector v, VectorContainer c, MaterializedField field,
                                            int recordCount, BufferAllocator allocator) {
     if (v != null) {
@@ -146,7 +145,6 @@ public class SchemaUtil {
         vectorMap.put(vvs[0].getField().getName(), vvs);
       } else {
         assert !isHyper;
-        @SuppressWarnings("resource")
         final ValueVector v = w.getValueVector();
         vectorMap.put(v.getField().getName(), v);
       }
@@ -169,7 +167,6 @@ public class SchemaUtil {
         }
         c.add(vvsOut);
       } else {
-        @SuppressWarnings("resource")
         final ValueVector v = (ValueVector) vectorMap.remove(field.getName());
         c.add(coerceVector(v, c, field, recordCount, allocator));
       }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/SimpleVectorWrapper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/SimpleVectorWrapper.java
index 4c41679..efd2d0b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/SimpleVectorWrapper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/SimpleVectorWrapper.java
@@ -78,7 +78,6 @@ public class SimpleVectorWrapper<T extends ValueVector> implements VectorWrapper
   }
 
 
-  @SuppressWarnings("resource")
   @Override
   public VectorWrapper<?> getChildWrapper(int[] ids) {
     if (ids.length == 1) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
index b77ece3..9e31ff8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
@@ -80,7 +80,6 @@ public class WritableBatch implements AutoCloseable {
         len += b.capacity();
       }
 
-      @SuppressWarnings("resource")
       DrillBuf newBuf = allocator.buffer(len);
       try {
         /* Copy data from each buffer into the compound buffer */
@@ -102,11 +101,8 @@ public class WritableBatch implements AutoCloseable {
 
         for (VectorWrapper<?> vv : container) {
           SerializedField fmd = fields.get(vectorIndex);
-          @SuppressWarnings("resource")
           ValueVector v = vv.getValueVector();
-          @SuppressWarnings("resource")
           DrillBuf bb = newBuf.slice(bufferOffset, fmd.getBufferLength());
-//        v.load(fmd, cbb.slice(bufferOffset, fmd.getBufferLength()));
           v.load(fmd, bb);
           vectorIndex++;
           bufferOffset += fmd.getBufferLength();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/AbstractColumnMetadata.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/AbstractColumnMetadata.java
index 00077bd..32be4f8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/AbstractColumnMetadata.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/AbstractColumnMetadata.java
@@ -27,8 +27,8 @@ import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.metadata.schema.parser.SchemaExprParser;
+import org.joda.time.format.DateTimeFormatter;
 
-import java.time.format.DateTimeFormatter;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.stream.Collectors;
@@ -266,7 +266,7 @@ public abstract class AbstractColumnMetadata extends AbstractPropertied implemen
         .append("[")
         .append(getClass().getSimpleName())
         .append(" ")
-        .append(schema().toString());
+        .append(schema().toString(false));
     if (variantSchema() != null) {
       buf.append(", variant: ")
          .append(variantSchema().toString());
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/MapColumnMetadata.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/MapColumnMetadata.java
index 3afc4d2..8b58bd0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/MapColumnMetadata.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/MapColumnMetadata.java
@@ -137,5 +137,4 @@ public class MapColumnMetadata extends AbstractColumnMetadata {
     }
     return builder.toString();
   }
-
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/PrimitiveColumnMetadata.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/PrimitiveColumnMetadata.java
index 4be120c..2187ffd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/PrimitiveColumnMetadata.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/PrimitiveColumnMetadata.java
@@ -22,15 +22,15 @@ import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.expr.TypeHelper;
 import org.apache.drill.exec.record.MaterializedField;
+import org.joda.time.Instant;
+import org.joda.time.LocalDate;
+import org.joda.time.LocalTime;
 import org.joda.time.Period;
+import org.joda.time.format.DateTimeFormat;
+import org.joda.time.format.DateTimeFormatter;
+import org.joda.time.format.ISODateTimeFormat;
 
 import java.math.BigDecimal;
-import java.time.LocalDate;
-import java.time.LocalTime;
-import java.time.ZoneOffset;
-import java.time.ZonedDateTime;
-import java.time.format.DateTimeFormatter;
-import java.time.format.DateTimeParseException;
 
 /**
  * Primitive (non-map) column. Describes non-nullable, nullable and array types
@@ -129,19 +129,19 @@ public class PrimitiveColumnMetadata extends AbstractColumnMetadata {
       switch (type) {
         case TIME:
           return formatValue == null
-            ? DateTimeFormatter.ISO_TIME.withZone(ZoneOffset.UTC) : DateTimeFormatter.ofPattern(formatValue);
+            ? ISODateTimeFormat.localTimeParser() : DateTimeFormat.forPattern(formatValue);
         case DATE:
           formatValue = format();
           return formatValue == null
-            ? DateTimeFormatter.ISO_DATE.withZone(ZoneOffset.UTC) : DateTimeFormatter.ofPattern(formatValue);
+            ? ISODateTimeFormat.localDateParser() : DateTimeFormat.forPattern(formatValue);
         case TIMESTAMP:
           formatValue = format();
           return formatValue == null
-            ? DateTimeFormatter.ISO_DATE_TIME.withZone(ZoneOffset.UTC) : DateTimeFormatter.ofPattern(formatValue);
+            ? ISODateTimeFormat.dateTimeNoMillis() : DateTimeFormat.forPattern(formatValue);
         default:
           throw new IllegalArgumentException("Column is not a date/time type: " + type.toString());
       }
-    } catch (IllegalArgumentException | DateTimeParseException e) {
+    } catch (IllegalArgumentException e) {
       throw new IllegalArgumentException(String.format("The format \"%s\" is not valid for type %s",
           formatValue, type), e);
     }
@@ -256,7 +256,7 @@ public class PrimitiveColumnMetadata extends AbstractColumnMetadata {
         case DATE:
           return LocalDate.parse(value, dateTimeFormatter());
         case TIMESTAMP:
-          return ZonedDateTime.parse(value, dateTimeFormatter());
+          return Instant.parse(value, dateTimeFormatter());
         case INTERVAL:
         case INTERVALDAY:
         case INTERVALYEAR:
@@ -264,7 +264,7 @@ public class PrimitiveColumnMetadata extends AbstractColumnMetadata {
         default:
           throw new IllegalArgumentException("Unsupported conversion: " + type.toString());
       }
-    } catch (IllegalArgumentException | DateTimeParseException e) {
+    } catch (IllegalArgumentException e) {
       logger.warn("Error while parsing type {} default value {}", type, value, e);
       throw new IllegalArgumentException(String.format("The string \"%s\" is not valid for type %s",
           value, type), e);
@@ -284,20 +284,11 @@ public class PrimitiveColumnMetadata extends AbstractColumnMetadata {
     }
     switch (type) {
       case TIME:
-        String formatValue = format();
-        DateTimeFormatter timeFormatter = formatValue == null
-          ? DateTimeFormatter.ISO_TIME.withZone(ZoneOffset.UTC) : DateTimeFormatter.ofPattern(formatValue);
-        return timeFormatter.format((LocalTime) value);
+        return dateTimeFormatter().print((LocalTime) value);
       case DATE:
-        formatValue = format();
-        DateTimeFormatter dateFormatter = formatValue == null
-          ? DateTimeFormatter.ISO_DATE.withZone(ZoneOffset.UTC) : DateTimeFormatter.ofPattern(formatValue);
-        return dateFormatter.format((LocalDate) value);
+        return dateTimeFormatter().print((LocalDate) value);
       case TIMESTAMP:
-        formatValue = format();
-        DateTimeFormatter dateTimeFormatter = formatValue == null
-          ? DateTimeFormatter.ISO_DATE_TIME.withZone(ZoneOffset.UTC) : DateTimeFormatter.ofPattern(formatValue);
-        return dateTimeFormatter.format((ZonedDateTime) value);
+        return dateTimeFormatter().print((Instant) value);
       default:
        return value.toString();
     }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/SchemaBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/SchemaBuilder.java
index 0c7c524..8b7f904 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/SchemaBuilder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/SchemaBuilder.java
@@ -133,6 +133,11 @@ public class SchemaBuilder implements SchemaContainer {
     return this;
   }
 
+  public SchemaBuilder add(String name, MinorType type, int precision, int scale) {
+    tupleBuilder.addDecimal(name, type, DataMode.REQUIRED, precision, scale);
+    return this;
+  }
+
   public SchemaBuilder addNullable(String name, MinorType type) {
     tupleBuilder.addNullable(name,  type);
     return this;
@@ -143,6 +148,11 @@ public class SchemaBuilder implements SchemaContainer {
     return this;
   }
 
+  public SchemaBuilder addNullable(String name, MinorType type, int precision, int scale) {
+    tupleBuilder.addDecimal(name, type, DataMode.OPTIONAL, precision, scale);
+    return this;
+  }
+
   public SchemaBuilder addArray(String name, MinorType type) {
     tupleBuilder.addArray(name, type);
     return this;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/TupleSchema.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/TupleSchema.java
index 7e90ff8..548aa30 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/TupleSchema.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/TupleSchema.java
@@ -176,6 +176,10 @@ public class TupleSchema extends AbstractPropertied implements TupleMetadata {
     return new BatchSchema(svMode, toFieldList());
   }
 
+  public static BatchSchema toBatchSchema(TupleMetadata schema) {
+    return ((TupleSchema) schema).toBatchSchema(SelectionVectorMode.NONE);
+  }
+
   @Override
   public String fullName(int index) {
     return fullName(metadata(index));
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
index 8906176..ed44c3e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
@@ -145,7 +145,6 @@ public class Drillbit implements AutoCloseable {
     this(config, SystemOptionManager.createDefaultOptionDefinitions(), serviceSet, classpathScan);
   }
 
-  @SuppressWarnings("resource")
   @VisibleForTesting
   public Drillbit(
     final DrillConfig config,
@@ -211,7 +210,6 @@ public class Drillbit implements AutoCloseable {
     }
     DrillbitEndpoint md = engine.start();
     manager.start(md, engine.getController(), engine.getDataConnectionCreator(), coord, storeProvider, profileStoreProvider);
-    @SuppressWarnings("resource")
     final DrillbitContext drillbitContext = manager.getContext();
     storageRegistry = drillbitContext.getStorage();
     storageRegistry.init();
@@ -335,7 +333,6 @@ public class Drillbit implements AutoCloseable {
       return;
     }
 
-    @SuppressWarnings("resource")
     final SystemOptionManager optionManager = getContext().getOptionManager();
 
     // parse out the properties, validate, and then set them
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/RemoteServiceSet.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/RemoteServiceSet.java
index 4133df9..aeb4475 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/RemoteServiceSet.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/RemoteServiceSet.java
@@ -40,7 +40,6 @@ public class RemoteServiceSet implements AutoCloseable {
     coordinator.close();
   }
 
-  @SuppressWarnings("resource")
   public static RemoteServiceSet getLocalServiceSet() {
     return new RemoteServiceSet(new LocalClusterCoordinator());
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRestServer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRestServer.java
index da8ccdc..344db1d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRestServer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRestServer.java
@@ -174,7 +174,6 @@ public class DrillRestServer extends ResourceConfig {
     @Inject
     EventExecutor executor;
 
-    @SuppressWarnings("resource")
     @Override
     public WebUserConnection provide() {
       final HttpSession session = request.getSession();
@@ -251,7 +250,6 @@ public class DrillRestServer extends ResourceConfig {
     @Inject
     EventExecutor executor;
 
-    @SuppressWarnings("resource")
     @Override
     public WebUserConnection provide() {
       final DrillbitContext drillbitContext = workManager.getContext();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRoot.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRoot.java
index 59f9688..e3dad27 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRoot.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRoot.java
@@ -80,7 +80,6 @@ public class DrillRoot {
   }
 
 
-  @SuppressWarnings("resource")
   @GET
   @Path("/state")
   @Produces(MediaType.APPLICATION_JSON)
@@ -93,7 +92,6 @@ public class DrillRoot {
     return setResponse(drillStatusMap);
   }
 
-  @SuppressWarnings("resource")
   @GET
   @Path("/gracePeriod")
   @Produces(MediaType.APPLICATION_JSON)
@@ -105,7 +103,6 @@ public class DrillRoot {
     return gracePeriodMap;
   }
 
-  @SuppressWarnings("resource")
   @GET
   @Path("/portNum")
   @Produces(MediaType.APPLICATION_JSON)
@@ -118,7 +115,6 @@ public class DrillRoot {
     return portMap;
   }
 
-  @SuppressWarnings("resource")
   @GET
   @Path("/queriesCount")
   @Produces(MediaType.APPLICATION_JSON)
@@ -127,7 +123,6 @@ public class DrillRoot {
     return setResponse(queriesInfo);
   }
 
-  @SuppressWarnings("resource")
   @POST
   @Path("/gracefulShutdown")
   @Produces(MediaType.APPLICATION_JSON)
@@ -137,7 +132,6 @@ public class DrillRoot {
     return shutdown(resp);
   }
 
-  @SuppressWarnings("resource")
   @POST
   @Path("/shutdown")
   @Produces(MediaType.APPLICATION_JSON)
@@ -148,7 +142,6 @@ public class DrillRoot {
     return shutdown(resp);
   }
 
-  @SuppressWarnings("resource")
   @POST
   @Path("/quiescent")
   @Produces(MediaType.APPLICATION_JSON)
@@ -159,7 +152,6 @@ public class DrillRoot {
     return shutdown(resp);
   }
 
-  @SuppressWarnings("resource")
   @GET
   @Path("/cluster.json")
   @Produces(MediaType.APPLICATION_JSON)
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/StatusResources.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/StatusResources.java
index 9123b65..f57a8b5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/StatusResources.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/StatusResources.java
@@ -88,7 +88,6 @@ public class StatusResources {
     return ViewableWithPermissions.create(authEnabled.get(), "/rest/status.ftl", sc, getStatusJSON());
   }
 
-  @SuppressWarnings("resource")
   private List<OptionWrapper> getSystemOptionsJSONHelper(boolean internal)
   {
     List<OptionWrapper> options = new LinkedList<>();
@@ -155,7 +154,6 @@ public class StatusResources {
     return getSystemOptionsHelper(true, uriInfo);
   }
 
-  @SuppressWarnings("resource")
   @POST
   @Path("option/{optionName}")
   @RolesAllowed(DrillUserPrincipal.ADMIN_ROLE)
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/StorageResources.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/StorageResources.java
index 8d71bdb..04090cc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/StorageResources.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/StorageResources.java
@@ -99,7 +99,6 @@ public class StorageResources {
     return ViewableWithPermissions.create(authEnabled.get(), "/rest/storage/list.ftl", sc, list);
   }
 
-  @SuppressWarnings("resource")
   @GET
   @Path("/storage/{name}.json")
   @Produces(MediaType.APPLICATION_JSON)
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServer.java
index cdde4aa..3fa2f3c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServer.java
@@ -157,7 +157,6 @@ public class WebServer implements AutoCloseable {
   /**
    * Start the web server including setup.
    */
-  @SuppressWarnings("resource")
   public void start() throws Exception {
     if (!config.getBoolean(ExecConstants.HTTP_ENABLE)) {
       return;
@@ -298,7 +297,6 @@ public class WebServer implements AutoCloseable {
         }
 
         // Clear all the resources allocated for this session
-        @SuppressWarnings("resource")
         final WebSessionResources webSessionResources =
             (WebSessionResources) session.getAttribute(WebSessionResources.class.getSimpleName());
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileResources.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileResources.java
index 86e3dde..1819962 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileResources.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileResources.java
@@ -231,7 +231,6 @@ public class ProfileResources {
   //max Param to cap listing of profiles
   private static final String MAX_QPROFILES_PARAM = "max";
 
-  @SuppressWarnings("resource")
   @GET
   @Path("/profiles.json")
   @Produces(MediaType.APPLICATION_JSON)
@@ -313,7 +312,6 @@ public class ProfileResources {
     return ViewableWithPermissions.create(authEnabled.get(), "/rest/profile/list.ftl", sc, profiles);
   }
 
-  @SuppressWarnings("resource")
   private QueryProfile getQueryProfile(String queryId) {
     QueryId id = QueryIdHelper.getQueryIdFromString(queryId);
 
@@ -385,7 +383,6 @@ public class ProfileResources {
     }
   }
 
-  @SuppressWarnings("resource")
   @GET
   @Path("/profiles/cancel/{queryid}")
   @Produces(MediaType.TEXT_PLAIN)
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/DrillFSDataInputStream.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/DrillFSDataInputStream.java
index ed14e82..46afd17 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/DrillFSDataInputStream.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/DrillFSDataInputStream.java
@@ -44,7 +44,6 @@ public class DrillFSDataInputStream extends FSDataInputStream {
     this(in, operatorStats, null);
   }
 
-  @SuppressWarnings("resource")
   public DrillFSDataInputStream(FSDataInputStream in, OperatorStats operatorStats,
       OpenFileTracker openFileTracker) throws IOException {
     super(new WrappedInputStream(in, operatorStats));
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemSchemaFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemSchemaFactory.java
index 45dea13..98d46a1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemSchemaFactory.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemSchemaFactory.java
@@ -69,7 +69,6 @@ public class FileSystemSchemaFactory extends AbstractSchemaFactory {
 
   @Override
   public void registerSchemas(SchemaConfig schemaConfig, SchemaPlus parent) throws IOException {
-    @SuppressWarnings("resource")
     FileSystemSchema schema = new FileSystemSchema(getName(), schemaConfig);
     SchemaPlus plusOfThis = parent.add(schema.getName(), schema);
     schema.setPlus(plusOfThis);
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 a1d1a88..a16edaf 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
@@ -42,9 +42,9 @@ import org.apache.drill.exec.physical.impl.WriterRecordBatch;
 import org.apache.drill.exec.physical.impl.StatisticsWriterRecordBatch;
 import org.apache.drill.exec.physical.impl.protocol.OperatorRecordBatch;
 import org.apache.drill.exec.physical.impl.scan.ScanOperatorExec;
-import org.apache.drill.exec.physical.impl.scan.file.BaseFileScanFramework;
 import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
 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.planner.physical.PlannerSettings;
 import org.apache.drill.exec.record.CloseableRecordBatch;
 import org.apache.drill.exec.record.RecordBatch;
@@ -244,11 +244,16 @@ public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements
       // Assemble the scan operator and its wrapper.
 
       try {
-        final BaseFileScanFramework<?> framework = buildFramework(scan);
+        final FileScanBuilder builder = frameworkBuilder(scan);
+        builder.setProjection(scan.getColumns());
+        builder.setFiles(scan.getWorkUnits());
+        builder.setConfig(plugin.easyConfig().fsConf);
         final Path selectionRoot = scan.getSelectionRoot();
         if (selectionRoot != null) {
-          framework.setSelectionRoot(selectionRoot, scan.getPartitionDepth());
+          builder.metadataOptions().setSelectionRoot(selectionRoot);
+          builder.metadataOptions().setPartitionDepth(scan.getPartitionDepth());
         }
+        FileScanFramework framework = builder.buildFileFramework();
         return new OperatorRecordBatch(
             context, scan,
             new ScanOperatorExec(
@@ -275,7 +280,7 @@ public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements
      * potentially many files
      * @throws ExecutionSetupException for all setup failures
      */
-    protected abstract BaseFileScanFramework<?> buildFramework(
+    protected abstract FileScanBuilder frameworkBuilder(
         EasySubScan scan) throws ExecutionSetupException;
   }
 
@@ -296,14 +301,12 @@ public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements
     }
 
     @Override
-    protected FileScanFramework buildFramework(EasySubScan scan) {
-
-      final FileScanFramework framework = new FileScanFramework(
-              scan.getColumns(),
-              scan.getWorkUnits(),
-              plugin.easyConfig().fsConf,
-              readerCreator);
-      return framework;
+    protected FileScanBuilder frameworkBuilder(
+        EasySubScan scan) throws ExecutionSetupException {
+
+      FileScanBuilder builder = new FileScanBuilder();
+      builder.setReaderFactory(readerCreator);
+      return builder;
     }
   }
 
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 e162d2a..fe0cbf5 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
@@ -22,6 +22,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.logical.FormatPluginConfig;
 import org.apache.drill.common.logical.StoragePluginConfig;
@@ -34,9 +35,10 @@ import org.apache.drill.exec.physical.base.AbstractGroupScan;
 import org.apache.drill.exec.physical.base.ScanStats;
 import org.apache.drill.exec.physical.base.ScanStats.GroupScanProperty;
 import org.apache.drill.exec.planner.common.DrillStatsTable.TableStatistics;
-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.columns.ColumnsScanFramework.FileReaderCreator;
+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.planner.physical.PlannerSettings;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
@@ -72,8 +74,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
 import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
 
-public class TextFormatPlugin extends EasyFormatPlugin<TextFormatPlugin.TextFormatConfig>
-      implements FileReaderCreator {
+public class TextFormatPlugin extends EasyFormatPlugin<TextFormatPlugin.TextFormatConfig> {
   private final static String PLUGIN_NAME = "text";
 
   @JsonTypeName(PLUGIN_NAME)
@@ -176,30 +177,48 @@ public class TextFormatPlugin extends EasyFormatPlugin<TextFormatPlugin.TextForm
     }
   }
 
-  public static class TextScanBatchCreator extends ScanFrameworkCreator {
+  /**
+   * Builds the readers for the V3 text scan operator.
+   */
+  private static class ColumnsReaderFactory extends FileReaderFactory {
+
+    private final TextFormatPlugin plugin;
+
+    public ColumnsReaderFactory(TextFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+
+    @Override
+    public ManagedReader<? extends FileSchemaNegotiator> newReader(
+        FileSplit split) {
+      TextParsingSettingsV3 settings = new TextParsingSettingsV3();
+      settings.set(plugin.getConfig());
+      return new CompliantTextBatchReader(split, fileSystem(), settings);
+    }
+  }
+
+  /**
+   * Builds the V3 text scan operator.
+   */
+  private static class TextScanBatchCreator extends ScanFrameworkCreator {
 
-    private final FileReaderCreator readerCreator;
     private final TextFormatPlugin textPlugin;
 
-    public TextScanBatchCreator(TextFormatPlugin plugin,
-        FileReaderCreator readerCreator) {
+    public TextScanBatchCreator(TextFormatPlugin plugin) {
       super(plugin);
-      this.readerCreator = readerCreator;
       textPlugin = plugin;
     }
 
     @Override
-    protected ColumnsScanFramework buildFramework(EasySubScan scan) {
-      ColumnsScanFramework framework = new ColumnsScanFramework(
-              scan.getColumns(),
-              scan.getWorkUnits(),
-              plugin.easyConfig().fsConf,
-              readerCreator);
+    protected FileScanBuilder frameworkBuilder(
+        EasySubScan scan) throws ExecutionSetupException {
+      ColumnsScanBuilder builder = new ColumnsScanBuilder();
+      builder.setReaderFactory(new ColumnsReaderFactory(textPlugin));
 
       // If this format has no headers, or wants to skip them,
       // then we must use the columns column to hold the data.
 
-      framework.requireColumnsArray(
+      builder.requireColumnsArray(
           ! textPlugin.getConfig().isHeaderExtractionEnabled());
 
       // Text files handle nulls in an unusual way. Missing columns
@@ -208,13 +227,17 @@ public class TextFormatPlugin extends EasyFormatPlugin<TextFormatPlugin.TextForm
       // columns from empty columns, but that is how CSV and other text
       // files have been defined within Drill.
 
-      framework.setNullType(
+      builder.setNullType(
           MajorType.newBuilder()
             .setMinorType(MinorType.VARCHAR)
             .setMode(DataMode.REQUIRED)
             .build());
 
-      return framework;
+      // Pass along the output schema, if any
+
+      builder.setOutputSchema(scan.getSchema());
+
+      return builder;
     }
   }
 
@@ -265,7 +288,7 @@ public class TextFormatPlugin extends EasyFormatPlugin<TextFormatPlugin.TextForm
     // readers provide identical functionality for the user; only the
     // internals differ.
     if (options.getBoolean(ExecConstants.ENABLE_V3_TEXT_READER_KEY)) {
-      return new TextScanBatchCreator(this, this);
+      return new TextScanBatchCreator(this);
     } else {
       return new ClassicScanBatchCreator(this);
     }
@@ -310,12 +333,6 @@ public class TextFormatPlugin extends EasyFormatPlugin<TextFormatPlugin.TextForm
   }
 
   @Override
-  public ManagedReader<ColumnsSchemaNegotiator> makeBatchReader(DrillFileSystem dfs, FileSplit split) {
-    TextParsingSettingsV3 settings = new TextParsingSettingsV3();
-    settings.set(getConfig());
-    return new CompliantTextBatchReader(split, dfs, settings);
-  }
-  @Override
   public boolean supportsStatistics() {
     return false;
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/compliant/RepeatedVarCharOutput.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/compliant/RepeatedVarCharOutput.java
index fc332b0..c8a681e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/compliant/RepeatedVarCharOutput.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/compliant/RepeatedVarCharOutput.java
@@ -185,7 +185,6 @@ public class RepeatedVarCharOutput extends TextOutput {
   }
 
   private void loadRepeatedOffsetAddress(){
-    @SuppressWarnings("resource")
     DrillBuf buf = vector.getOffsetVector().getBuffer();
     checkBuf(buf);
     this.repeatedOffset = buf.memoryAddress() + 4;
@@ -194,7 +193,6 @@ public class RepeatedVarCharOutput extends TextOutput {
   }
 
   private void loadVarCharDataAddress(){
-    @SuppressWarnings("resource")
     DrillBuf buf = vector.getDataVector().getBuffer();
     checkBuf(buf);
     this.characterData = buf.memoryAddress();
@@ -203,7 +201,6 @@ public class RepeatedVarCharOutput extends TextOutput {
   }
 
   private void loadVarCharOffsetAddress(){
-    @SuppressWarnings("resource")
     DrillBuf buf = vector.getDataVector().getOffsetVector().getBuffer();
     checkBuf(buf);
     this.charLengthOffset = buf.memoryAddress() + 4;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/FixedWidthRepeatedReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/FixedWidthRepeatedReader.java
index 0b2d678..29b60ed 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/FixedWidthRepeatedReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/FixedWidthRepeatedReader.java
@@ -142,7 +142,6 @@ public class FixedWidthRepeatedReader extends VarLengthColumn<RepeatedValueVecto
     }
   }
 
-  @SuppressWarnings("resource")
   @Override
   protected boolean readAndStoreValueSizeInformation() {
     int numLeftoverVals = 0;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetColumnMetadata.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetColumnMetadata.java
index 9c2444c..62b3b56 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetColumnMetadata.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetColumnMetadata.java
@@ -142,7 +142,6 @@ public class ParquetColumnMetadata {
         column, columnChunkMetaData, vector, se);
   }
 
-  @SuppressWarnings("resource")
   FixedWidthRepeatedReader makeRepeatedFixedWidthReader(ParquetRecordReader reader) throws Exception {
     final RepeatedValueVector repeatedVector = RepeatedValueVector.class.cast(vector);
     ColumnReader<?> dataReader = ColumnReaderFactory.createFixedColumnReader(reader, true,
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet2/DrillParquetGroupConverter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet2/DrillParquetGroupConverter.java
index 58a4f58..296d34c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet2/DrillParquetGroupConverter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet2/DrillParquetGroupConverter.java
@@ -161,7 +161,6 @@ public class DrillParquetGroupConverter extends GroupConverter {
     }
   }
 
-  @SuppressWarnings("resource")
   private PrimitiveConverter getConverterForType(String name, PrimitiveType type) {
 
     switch(type.getPrimitiveTypeName()) {
@@ -603,7 +602,6 @@ public class DrillParquetGroupConverter extends GroupConverter {
     private VarBinaryWriter writer;
     private VarBinaryHolder holder = new VarBinaryHolder();
 
-    @SuppressWarnings("resource")
     public DrillFixedBinaryToVarbinaryConverter(VarBinaryWriter writer, int length, DrillBuf buf) {
       this.writer = writer;
       holder.buffer = buf = buf.reallocIfNeeded(length);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/FunctionsIterator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/FunctionsIterator.java
index 63798a2..033e7c8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/FunctionsIterator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/FunctionsIterator.java
@@ -45,7 +45,6 @@ public class FunctionsIterator implements Iterator<Object> {
     FunctionLookupContext functionLookupContext = context.getFunctionRegistry();
     //Check true instance type
     if (functionLookupContext instanceof FunctionImplementationRegistry) {
-      @SuppressWarnings("resource")
       FunctionImplementationRegistry functionImplRegistry = (FunctionImplementationRegistry) functionLookupContext;
       Map<String, List<FunctionHolder>> jarFunctionListMap = functionImplRegistry.getAllJarsWithFunctionsHolders();
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/util/StoragePluginTestUtils.java b/exec/java-exec/src/main/java/org/apache/drill/exec/util/StoragePluginTestUtils.java
index 1c7b2ab..71c7806 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/util/StoragePluginTestUtils.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/util/StoragePluginTestUtils.java
@@ -66,7 +66,6 @@ public class StoragePluginTestUtils {
                                           final StoragePluginRegistry pluginRegistry,
                                           final File tmpDirPath,
                                           String... schemas) throws ExecutionSetupException {
-    @SuppressWarnings("resource")
     final FileSystemPlugin plugin = (FileSystemPlugin) pluginRegistry.getPlugin(pluginName);
     final FileSystemConfig pluginConfig = (FileSystemConfig) plugin.getConfig();
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/JsonReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/JsonReader.java
index 34b1f80..3426243 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/JsonReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/JsonReader.java
@@ -152,9 +152,6 @@ public class JsonReader extends BaseJsonProcessor {
     }
   }
 
-
-
-  @SuppressWarnings("resource")
   @Override
   public void ensureAtLeastOneField(ComplexWriter writer) {
     JsonReaderUtils.ensureAtLeastOneField(writer, columns, allTextMode, emptyArrayWriters);
@@ -182,7 +179,6 @@ public class JsonReader extends BaseJsonProcessor {
     setSource(data.getBytes(Charsets.UTF_8));
   }
 
-  @SuppressWarnings("resource")
   public void setSource(byte[] bytes) throws IOException {
     setSource(new SeekableBAIS(bytes));
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/VectorOutput.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/VectorOutput.java
index 9f0ac8a..b65e6c2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/VectorOutput.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/VectorOutput.java
@@ -301,7 +301,6 @@ abstract class VectorOutput {
       return innerRun();
     }
 
-    @SuppressWarnings("resource")
     @Override
     public void writeBinary(boolean isNull) throws IOException {
       VarBinaryWriter bin = writer.varBinary(fieldName);
@@ -334,7 +333,6 @@ abstract class VectorOutput {
 
     @Override
     public void writeTime(boolean isNull) throws IOException {
-      @SuppressWarnings("resource")
       TimeWriter t = writer.time(fieldName);
       if(!isNull){
         LocalTime localTime = OffsetTime.parse(parser.getValueAsString(), DateUtility.isoFormatTime).toLocalTime();
@@ -342,7 +340,6 @@ abstract class VectorOutput {
       }
     }
 
-    @SuppressWarnings("resource")
     @Override
     public void writeTimestamp(boolean isNull) throws IOException {
       TimeStampWriter ts = writer.timeStamp(fieldName);
@@ -379,7 +376,6 @@ abstract class VectorOutput {
 
     @Override
     public void writeInteger(boolean isNull) throws IOException {
-      @SuppressWarnings("resource")
       BigIntWriter intWriter = writer.bigInt(fieldName);
       if(!isNull){
         intWriter.writeBigInt(Long.parseLong(parser.getValueAsString()));
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/VectorContainerWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/VectorContainerWriter.java
index 958781b..5866a95 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/VectorContainerWriter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/VectorContainerWriter.java
@@ -101,7 +101,6 @@ public class VectorContainerWriter extends AbstractFieldWriter implements Comple
       super("", null, callback);
     }
 
-    @SuppressWarnings("resource")
     @Override
     public <T extends ValueVector> T addOrGet(String name, MajorType type, Class<T> clazz) {
       try {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentsRunner.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentsRunner.java
index a7c7263..6c49d6e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentsRunner.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentsRunner.java
@@ -210,7 +210,6 @@ public class FragmentsRunner {
    */
   private void sendRemoteFragments(final DrillbitEndpoint assignment, final Collection<PlanFragment> fragments,
                                    final CountDownLatch latch, final FragmentSubmitFailures fragmentSubmitFailures) {
-    @SuppressWarnings("resource")
     final Controller controller = drillbitContext.getController();
     final InitializeFragments.Builder fb = InitializeFragments.newBuilder();
     for(final PlanFragment planFragment : fragments) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java
index a5c4d42..d6a9bd9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java
@@ -199,7 +199,6 @@ public class QueryManager implements AutoCloseable {
    * (3) Leaf fragment: running, send the cancel signal through a tunnel. The cancel is done directly.
    */
   void cancelExecutingFragments(final DrillbitContext drillbitContext) {
-    @SuppressWarnings("resource")
     final Controller controller = drillbitContext.getController();
     for(final FragmentData data : fragmentDataSet) {
       switch(data.getState()) {
@@ -228,7 +227,6 @@ public class QueryManager implements AutoCloseable {
    * sending any message. Resume all fragments through the control tunnel.
    */
   void unpauseExecutingFragments(final DrillbitContext drillbitContext) {
-    @SuppressWarnings("resource")
     final Controller controller = drillbitContext.getController();
     for(final FragmentData data : fragmentDataSet) {
       final DrillbitEndpoint endpoint = data.getEndpoint();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/DistributedQueryQueue.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/DistributedQueryQueue.java
index 73e8de6..58d8416 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/DistributedQueryQueue.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/DistributedQueryQueue.java
@@ -240,7 +240,6 @@ public class DistributedQueryQueue implements QueryQueue {
    * queue
    */
 
-  @SuppressWarnings("resource")
   @Override
   public QueueLease enqueue(QueryId queryId, double cost) throws QueryQueueException, QueueTimeoutException {
     final String queueName;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/DynamicResourceManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/DynamicResourceManager.java
index 473401f..7491e60 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/DynamicResourceManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/DynamicResourceManager.java
@@ -79,7 +79,6 @@ public class DynamicResourceManager implements ResourceManager {
       return;
     }
     nextUpdateTime = now + recheckDelayMs;
-    @SuppressWarnings("resource")
     SystemOptionManager systemOptions = context.getOptionManager();
     if (systemOptions.getOption(ExecConstants.ENABLE_QUEUE)) {
       if (queueingRm == null) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/ResourceManagerBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/ResourceManagerBuilder.java
index f4e460d..ec1f2c1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/ResourceManagerBuilder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/ResourceManagerBuilder.java
@@ -59,7 +59,6 @@ public class ResourceManagerBuilder {
     this.context = context;
   }
 
-  @SuppressWarnings("resource")
   public ResourceManager build() {
     ClusterCoordinator coord = context.getClusterCoordinator();
     DrillConfig config = context.getConfig();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java
index e374c01..9e9bf3f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java
@@ -238,7 +238,6 @@ public class FragmentExecutor implements Runnable {
     eventProcessor.receiverFinished(handle);
   }
 
-  @SuppressWarnings("resource")
   @Override
   public void run() {
     final Thread myThread = Thread.currentThread();
@@ -482,17 +481,21 @@ public class FragmentExecutor implements Runnable {
   }
 
   private class ExecutorStateImpl implements FragmentContext.ExecutorState {
+    @Override
     public boolean shouldContinue() {
       return FragmentExecutor.this.shouldContinue();
     }
 
+    @Override
     public void fail(final Throwable t) {
       FragmentExecutor.this.fail(t);
     }
 
+    @Override
     public boolean isFailed() {
       return fragmentState.get() == FragmentState.FAILED;
     }
+    @Override
     public Throwable getFailureCause(){
       return deferredException.getException();
     }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/user/PlanSplitter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/user/PlanSplitter.java
index e61814a..e907c54 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/user/PlanSplitter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/user/PlanSplitter.java
@@ -61,7 +61,6 @@ public class PlanSplitter {
    * @param connection
    * @return
    */
-  @SuppressWarnings("resource")
   public QueryPlanFragments planFragments(DrillbitContext dContext, QueryId queryId,
       GetQueryPlanFragments req, UserClientConnection connection) {
     QueryPlanFragments.Builder responseBuilder = QueryPlanFragments.newBuilder();
diff --git a/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java b/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java
index d9e9bc0..6b6c0f0 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java
@@ -88,7 +88,6 @@ public class PlanningBase extends ExecTest {
     final LogicalPlanPersistence logicalPlanPersistence = new LogicalPlanPersistence(config, scanResult);
     final SystemOptionManager systemOptions = new SystemOptionManager(logicalPlanPersistence, provider, config);
     systemOptions.init();
-    @SuppressWarnings("resource")
     final UserSession userSession = UserSession.Builder.newBuilder().withOptionManager(systemOptions).build();
     final SessionOptionManager sessionOptions = userSession.getOptions();
     final QueryOptionManager queryOptions = new QueryOptionManager(sessionOptions);
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestSchemaCommands.java b/exec/java-exec/src/test/java/org/apache/drill/TestSchemaCommands.java
index 7391366..84acd1a 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestSchemaCommands.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestSchemaCommands.java
@@ -30,6 +30,7 @@ import org.apache.drill.test.ClusterFixture;
 import org.apache.drill.test.ClusterFixtureBuilder;
 import org.apache.drill.test.ClusterTest;
 import org.apache.hadoop.fs.Path;
+import org.joda.time.LocalDate;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
@@ -39,7 +40,6 @@ import org.junit.rules.ExpectedException;
 import java.io.File;
 import java.nio.file.Files;
 import java.nio.file.Paths;
-import java.time.LocalDate;
 import java.util.Arrays;
 import java.util.LinkedHashMap;
 import java.util.Map;
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestWriteToDisk.java b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestWriteToDisk.java
index 8427b51..db7f743 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestWriteToDisk.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestWriteToDisk.java
@@ -85,7 +85,6 @@ public class TestWriteToDisk extends ExecTest {
         VectorContainer container = new VectorContainer();
         container.addCollection(vectorList);
         container.setRecordCount(4);
-        @SuppressWarnings("resource")
         WritableBatch batch = WritableBatch.getBatchNoHVWrap(
             container.getRecordCount(), container, false);
         VectorAccessibleSerializable wrap = new VectorAccessibleSerializable(
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java
index cd460ec..c03a469 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java
@@ -156,7 +156,6 @@ public class ExpressionInterpreterTest  extends PopUnitTestBase {
   }
 
   protected void doTest(String expressionStr, String[] colNames, TypeProtos.MajorType[] colTypes, String[] expectFirstTwoValues, BitControl.PlanFragment planFragment) throws Exception {
-    @SuppressWarnings("resource")
     final RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
     final Drillbit bit1 = new Drillbit(CONFIG, serviceSet);
 
@@ -178,7 +177,6 @@ public class ExpressionInterpreterTest  extends PopUnitTestBase {
 
     batch.next();
 
-    @SuppressWarnings("resource")
     final ValueVector vv = evalExprWithInterpreter(expressionStr, batch, bit1);
 
     // Verify the first 2 values in the output of evaluation.
@@ -194,7 +192,6 @@ public class ExpressionInterpreterTest  extends PopUnitTestBase {
     bit1.close();
   }
 
-  @SuppressWarnings("resource")
   private ScanBatch createMockScanBatch(Drillbit bit, MockSubScanPOP scanPOP, BitControl.PlanFragment planFragment) {
     final List<RecordBatch> children = Lists.newArrayList();
     final MockScanBatchCreator creator = new MockScanBatchCreator();
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestConvertFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestConvertFunctions.java
index 1bc5f55..5776d43 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestConvertFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestConvertFunctions.java
@@ -553,7 +553,6 @@ public class TestConvertFunctions extends BaseTestQuery {
   public void testHadooopVInt() throws Exception {
     final int _0 = 0;
     final int _9 = 9;
-    @SuppressWarnings("resource")
     final DrillBuf buffer = getAllocator().buffer(_9);
 
     long longVal = 0;
@@ -641,7 +640,6 @@ public class TestConvertFunctions extends BaseTestQuery {
     for(QueryDataBatch result : resultList) {
       if (result.getData() != null) {
         loader.load(result.getHeader().getDef(), result.getData());
-        @SuppressWarnings("resource")
         ValueVector v = loader.iterator().next().getValueVector();
         for (int j = 0; j < v.getAccessor().getValueCount(); j++) {
           if  (v instanceof VarCharVector) {
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFunctions.java
index 5b8ac75..0dcf84c 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFunctions.java
@@ -63,7 +63,6 @@ public class TestSimpleFunctions extends ExecTest {
 
   @Test
   public void testHashFunctionResolution() throws JClassAlreadyExistsException, IOException {
-    @SuppressWarnings("resource")
     final FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
     // test required vs nullable Int input
     resolveHash(new TypedNullConstant(Types.optional(TypeProtos.MinorType.INT)),
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java
index 3de912d..69798af 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java
@@ -47,7 +47,6 @@ public class TestMergingReceiver extends PopUnitTestBase {
 
   @Test
   public void twoBitTwoExchange() throws Exception {
-    @SuppressWarnings("resource")
     final RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
 
     try (final Drillbit bit1 = new Drillbit(CONFIG, serviceSet);
@@ -76,7 +75,6 @@ public class TestMergingReceiver extends PopUnitTestBase {
 
   @Test
   public void testMultipleProvidersMixedSizes() throws Exception {
-    @SuppressWarnings("resource")
     final RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
 
     try (final Drillbit bit1 = new Drillbit(CONFIG, serviceSet);
@@ -100,7 +98,6 @@ public class TestMergingReceiver extends PopUnitTestBase {
         count += batchRowCount;
         batchLoader.load(queryData.getDef(), b.getData());
         for (final VectorWrapper<?> vw : batchLoader) {
-          @SuppressWarnings("resource")
           final ValueVector vv = vw.getValueVector();
           final ValueVector.Accessor va = vv.getAccessor();
           final MaterializedField materializedField = vv.getField();
@@ -125,7 +122,6 @@ public class TestMergingReceiver extends PopUnitTestBase {
 
   @Test
   public void handleEmptyBatch() throws Exception {
-    @SuppressWarnings("resource")
     final RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
 
     try (final Drillbit bit1 = new Drillbit(CONFIG, serviceSet);
@@ -155,7 +151,6 @@ public class TestMergingReceiver extends PopUnitTestBase {
 
   @Test
   public void handleEmptyBatchNoSchema() throws Exception {
-    @SuppressWarnings("resource")
     final RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
 
     try (final Drillbit bit1 = new Drillbit(CONFIG, serviceSet);
@@ -185,7 +180,6 @@ public class TestMergingReceiver extends PopUnitTestBase {
 
   @Test
   public void testMultipleProvidersEmptyBatches() throws Exception {
-    @SuppressWarnings("resource")
     final RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
 
     try (final Drillbit bit1 = new Drillbit(CONFIG, serviceSet);
@@ -209,7 +203,6 @@ public class TestMergingReceiver extends PopUnitTestBase {
         count += batchRowCount;
         batchLoader.load(queryData.getDef(), b.getData());
         for (final VectorWrapper<?> vw : batchLoader) {
-          @SuppressWarnings("resource")
           final ValueVector vv = vw.getValueVector();
           final ValueVector.Accessor va = vv.getAccessor();
           final MaterializedField materializedField = vv.getField();
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
new file mode 100644
index 0000000..fe4a8b8
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/BaseScanOperatorExecTest.java
@@ -0,0 +1,180 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.scan;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedScanFramework.ScanFrameworkBuilder;
+import org.apache.drill.exec.physical.impl.scan.ScanTestUtils.ScanFixture;
+import org.apache.drill.exec.physical.impl.scan.ScanTestUtils.ScanFixtureBuilder;
+import org.apache.drill.exec.physical.impl.scan.framework.BasicScanFactory;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.impl.scan.framework.SchemaNegotiator;
+import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
+import org.apache.drill.exec.physical.rowSet.RowSetLoader;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.test.SubOperatorTest;
+import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
+import org.apache.drill.test.rowSet.RowSetUtilities;
+
+/**
+ * Test of the scan operator framework. Here the focus is on the
+ * implementation of the scan operator itself. This operator is
+ * based on a number of lower-level abstractions, each of which has
+ * its own unit tests. To make this more concrete: review the scan
+ * operator code paths. Each path should be exercised by one or more
+ * of the tests here. If, however, the code path depends on the
+ * details of another, supporting class, then tests for that class
+ * appear elsewhere.
+ */
+
+public class BaseScanOperatorExecTest extends SubOperatorTest {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BaseScanOperatorExecTest.class);
+
+  /**
+   * Base class for the "mock" readers used in this test. The mock readers
+   * follow the normal (enhanced) reader API, but instead of actually reading
+   * from a data source, they just generate data with a known schema.
+   * They also expose internal state such as identifying which methods
+   * were actually called.
+   */
+
+  protected static abstract class BaseMockBatchReader implements ManagedReader<SchemaNegotiator> {
+    protected boolean openCalled;
+    protected boolean closeCalled;
+    protected int startIndex;
+    protected int batchCount;
+    protected int batchLimit;
+    protected ResultSetLoader tableLoader;
+
+    protected void makeBatch() {
+      RowSetLoader writer = tableLoader.writer();
+      int offset = (batchCount - 1) * 20 + startIndex;
+      writeRow(writer, offset + 10, "fred");
+      writeRow(writer, offset + 20, "wilma");
+    }
+
+    protected void writeRow(RowSetLoader writer, int col1, String col2) {
+      writer.start();
+      if (writer.column(0) != null) {
+        writer.scalar(0).setInt(col1);
+      }
+      if (writer.column(1) != null) {
+        writer.scalar(1).setString(col2);
+      }
+      writer.save();
+    }
+
+    @Override
+    public void close() {
+      closeCalled = true;
+    }
+  }
+
+  /**
+   * Mock reader that pretends to have a schema at open time
+   * like an HBase or JDBC reader.
+   */
+
+  protected static class MockEarlySchemaReader extends BaseMockBatchReader {
+
+    @Override
+    public boolean open(SchemaNegotiator schemaNegotiator) {
+      openCalled = true;
+      TupleMetadata schema = new SchemaBuilder()
+          .add("a", MinorType.INT)
+          .addNullable("b", MinorType.VARCHAR, 10)
+          .buildSchema();
+      schemaNegotiator.setTableSchema(schema, true);
+      tableLoader = schemaNegotiator.build();
+      return true;
+    }
+
+    @Override
+    public boolean next() {
+      batchCount++;
+      if (batchCount > batchLimit) {
+        return false;
+      }
+
+      makeBatch();
+      return true;
+    }
+  }
+
+  protected SingleRowSet makeExpected() {
+    return makeExpected(0);
+  }
+
+  protected SingleRowSet makeExpected(int offset) {
+    BatchSchema expectedSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addNullable("b", MinorType.VARCHAR, 10)
+        .build();
+    SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addRow(offset + 10, "fred")
+        .addRow(offset + 20, "wilma")
+        .build();
+    return expected;
+  }
+
+  protected void verifyBatch(int offset, VectorContainer output) {
+    SingleRowSet expected = makeExpected(offset);
+    RowSetUtilities.verify(expected, fixture.wrap(output));
+  }
+
+  public static class BaseScanFixtureBuilder extends ScanFixtureBuilder {
+
+    public ScanFrameworkBuilder builder = new ScanFrameworkBuilder();
+    public final List<ManagedReader<? extends SchemaNegotiator>> readers = new ArrayList<>();
+
+    public BaseScanFixtureBuilder() {
+      super(fixture);
+    }
+
+    @Override
+    public ScanFrameworkBuilder builder() { return builder; }
+
+    public void addReader(ManagedReader<? extends SchemaNegotiator> reader) {
+      readers.add(reader);
+    }
+
+    @Override
+    protected ManagedScanFramework newFramework() {
+      builder.setReaderFactory(new BasicScanFactory(readers.iterator()));
+      return new ManagedScanFramework(builder);
+    }
+  }
+
+  @SafeVarargs
+  public static ScanFixture simpleFixture(ManagedReader<? extends SchemaNegotiator>...readers) {
+    BaseScanFixtureBuilder builder = new BaseScanFixtureBuilder();
+    builder.projectAll();
+    for (ManagedReader<? extends SchemaNegotiator> reader : readers) {
+      builder.addReader(reader);
+    }
+    return builder.build();
+  }
+
+}
\ No newline at end of file
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 9514e12..de7b6c2 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
@@ -22,19 +22,25 @@ import java.util.List;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.Types;
+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.PartitionColumn;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedScanFramework.ScanFrameworkBuilder;
 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.SchemaLevelProjection.SchemaProjectionResolver;
+import org.apache.drill.exec.physical.impl.scan.project.ReaderLevelProjection.ReaderProjectionResolver;
 import org.apache.drill.exec.physical.rowSet.impl.RowSetTestUtils;
 import org.apache.drill.exec.record.MaterializedField;
 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.shaded.guava.com.google.common.collect.ImmutableList;
+import org.apache.drill.test.OperatorFixture;
 
 import avro.shaded.com.google.common.collect.Lists;
 
@@ -48,6 +54,70 @@ public class ScanTestUtils {
   public static final String SUFFIX_COL = "suffix";
   public static final String PARTITION_COL = "dir";
 
+
+  public static abstract class ScanFixtureBuilder {
+
+    public final OperatorFixture opFixture;
+
+    public ScanFixtureBuilder(OperatorFixture opFixture) {
+      this.opFixture = opFixture;
+    }
+
+    public abstract ScanFrameworkBuilder builder();
+
+    public void projectAll() {
+      builder().setProjection(RowSetTestUtils.projectAll());
+    }
+
+    public void projectAllWithMetadata(int dirs) {
+      builder().setProjection(ScanTestUtils.projectAllWithMetadata(dirs));
+    }
+
+    public void setProjection(String... projCols) {
+      builder().setProjection(RowSetTestUtils.projectList(projCols));
+    }
+
+    public void setProjection(List<SchemaPath> projection) {
+      builder().setProjection(projection);
+    }
+
+    protected abstract ManagedScanFramework newFramework();
+
+    public ScanFixture build() {
+      ManagedScanFramework framework = newFramework();
+      ScanOperatorExec scanOp = new ScanOperatorExec(framework);
+      Scan scanConfig = new AbstractSubScan("bob") {
+
+        @Override
+        public int getOperatorType() {
+          return 0;
+        }
+      };
+      OperatorContext opContext = opFixture.newOperatorContext(scanConfig);
+      scanOp.bind(opContext);
+      return new ScanFixture(opContext, scanOp);
+    }
+  }
+
+  public static class ScanFixture {
+
+    private OperatorContext opContext;
+    public ScanOperatorExec scanOp;
+
+    public ScanFixture(OperatorContext opContext, ScanOperatorExec scanOp) {
+      this.opContext = opContext;
+      this.scanOp = scanOp;
+    }
+
+    public void close() {
+      try {
+        scanOp.close();
+      } finally {
+        opContext.close();
+      }
+    }
+  }
+
   /**
    * Type-safe way to define a list of parsers.
    * @param parsers as a varArgs list convenient for testing
@@ -59,7 +129,7 @@ public class ScanTestUtils {
     return ImmutableList.copyOf(parsers);
   }
 
-  public static List<SchemaProjectionResolver> resolvers(SchemaProjectionResolver... resolvers) {
+  public static List<ReaderProjectionResolver> resolvers(ReaderProjectionResolver... resolvers) {
     return ImmutableList.copyOf(resolvers);
   }
 
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 2d066de..8caece4 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
@@ -27,8 +27,10 @@ import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.physical.impl.scan.columns.ColumnsArrayManager;
 import org.apache.drill.exec.physical.impl.scan.file.FileMetadataManager;
+import org.apache.drill.exec.physical.impl.scan.file.FileMetadataManager.FileMetadataOptions;
 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.rowSet.ResultSetLoader;
 import org.apache.drill.exec.physical.rowSet.impl.RowSetTestUtils;
 import org.apache.drill.exec.record.metadata.SchemaBuilder;
@@ -55,6 +57,14 @@ public class TestColumnsArray extends SubOperatorTest {
     ResultSetLoader loader;
   }
 
+  private FileMetadataOptions standardOptions(Path filePath) {
+    FileMetadataOptions options = new FileMetadataOptions();
+    options.useLegacyWildcardExpansion(false); // Don't expand partition columns for wildcard
+    options.setSelectionRoot(new Path("hdfs:///w"));
+    options.setFiles(Lists.newArrayList(filePath));
+    return options;
+  }
+
   private MockScanner buildScanner(List<SchemaPath> projList) {
 
     MockScanner mock = new MockScanner();
@@ -64,11 +74,7 @@ public class TestColumnsArray extends SubOperatorTest {
     Path filePath = new Path("hdfs:///w/x/y/z.csv");
     FileMetadataManager metadataManager = new FileMetadataManager(
         fixture.getOptionManager(),
-        false, // Don't expand partition columns for wildcard
-        false, // N/A
-        new Path("hdfs:///w"),
-        FileMetadataManager.AUTO_PARTITION_DEPTH,
-        Lists.newArrayList(filePath));
+        standardOptions(filePath));
 
     // ...and the columns array manager
 
@@ -76,14 +82,15 @@ public class TestColumnsArray extends SubOperatorTest {
 
     // Configure the schema orchestrator
 
-    mock.scanner = new ScanSchemaOrchestrator(fixture.allocator());
-    mock.scanner.withMetadata(metadataManager);
-    mock.scanner.addParser(colsManager.projectionParser());
-    mock.scanner.addResolver(colsManager.resolver());
+    ScanOrchestratorBuilder builder = new ScanOrchestratorBuilder();
+    builder.withMetadata(metadataManager);
+    builder.addParser(colsManager.projectionParser());
+    builder.addResolver(colsManager.resolver());
 
     // SELECT <proj list> ...
 
-    mock.scanner.build(projList);
+    builder.setProjection(projList);
+    mock.scanner = new ScanSchemaOrchestrator(fixture.allocator(), builder);
 
     // FROM z.csv
 
@@ -252,12 +259,11 @@ public class TestColumnsArray extends SubOperatorTest {
 
     // Configure the schema orchestrator
 
-    ScanSchemaOrchestrator scanner = new ScanSchemaOrchestrator(fixture.allocator());
-    scanner.addParser(colsManager.projectionParser());
-    scanner.addResolver(colsManager.resolver());
-
-    scanner.build(cols);
-    return scanner;
+    ScanOrchestratorBuilder builder = new ScanOrchestratorBuilder();
+    builder.addParser(colsManager.projectionParser());
+    builder.addResolver(colsManager.resolver());
+    builder.setProjection(cols);
+    return new ScanSchemaOrchestrator(fixture.allocator(), builder);
   }
 
   /**
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestColumnsArrayFramework.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestColumnsArrayFramework.java
index 8c7e645..2095f04 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestColumnsArrayFramework.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestColumnsArrayFramework.java
@@ -28,22 +28,26 @@ import java.util.Iterator;
 import java.util.List;
 
 import org.apache.drill.categories.RowSetTests;
-import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos.MinorType;
-import org.apache.drill.exec.physical.impl.scan.TestFileScanFramework.BaseFileScanOpFixture;
+import org.apache.drill.exec.physical.impl.scan.ScanTestUtils.ScanFixture;
+import org.apache.drill.exec.physical.impl.scan.ScanTestUtils.ScanFixtureBuilder;
 import org.apache.drill.exec.physical.impl.scan.TestFileScanFramework.DummyFileWork;
 import org.apache.drill.exec.physical.impl.scan.columns.ColumnsArrayManager;
 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.columns.ColumnsSchemaNegotiator;
-import org.apache.drill.exec.physical.impl.scan.columns.ColumnsScanFramework.FileReaderCreator;
-import org.apache.drill.exec.physical.impl.scan.file.BaseFileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileReaderFactory;
+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.impl.scan.framework.ManagedScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedScanFramework.ScanFrameworkBuilder;
 import org.apache.drill.exec.physical.rowSet.impl.RowSetTestUtils;
 import org.apache.drill.exec.record.metadata.SchemaBuilder;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
-import org.apache.drill.exec.store.dfs.DrillFileSystem;
+import org.apache.drill.exec.store.dfs.easy.FileWork;
 import org.apache.drill.test.SubOperatorTest;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.FileSplit;
 import org.junit.Test;
@@ -60,30 +64,55 @@ public class TestColumnsArrayFramework extends SubOperatorTest {
 
   private static final Path MOCK_FILE_PATH = new Path("file:/w/x/y/z.csv");
 
-  public static class ColumnsScanOpFixture extends BaseFileScanOpFixture implements FileReaderCreator {
+  public static class MockFileReaderFactory extends FileReaderFactory {
+    public Iterator<DummyColumnsReader> readerIter;
 
-    protected final List<DummyColumnsReader> readers = new ArrayList<>();
-    protected Iterator<DummyColumnsReader> readerIter;
+    public MockFileReaderFactory(List<DummyColumnsReader> readers) {
+      readerIter = readers.iterator();
+    }
 
-    public void addReader(DummyColumnsReader reader) {
-      readers.add(reader);
-      files.add(new DummyFileWork(reader.filePath()));
+    @Override
+    public ManagedReader<? extends FileSchemaNegotiator> newReader(
+        FileSplit split) {
+      DummyColumnsReader reader = readerIter.next();
+      assert reader != null;
+      assert split.getPath().equals(reader.filePath());
+      return reader;
+    }
+  }
+
+  public static class ColumnsScanFixtureBuilder extends ScanFixtureBuilder {
+
+    public ColumnsScanBuilder builder = new ColumnsScanBuilder();
+    public List<DummyColumnsReader> readers = new ArrayList<>();
+
+    public ColumnsScanFixtureBuilder() {
+      super(fixture);
     }
 
     @Override
-    protected BaseFileScanFramework<?> buildFramework() {
-      readerIter = readers.iterator();
-      return new ColumnsScanFramework(projection, files, fsConfig, this);
+    public ScanFrameworkBuilder builder() { return builder; }
+
+    public void addReader(DummyColumnsReader reader) {
+      readers.add(reader);
     }
 
     @Override
-    public ManagedReader<ColumnsSchemaNegotiator> makeBatchReader(
-        DrillFileSystem dfs,
-        FileSplit split) throws ExecutionSetupException {
-      if (! readerIter.hasNext()) {
-        return null;
+    protected ManagedScanFramework newFramework() {
+
+      // Bass-ackward construction of the list of files from
+      // a set of text fixture readers. Normal implementations
+      // create readers from file splits, not the other way around
+      // as is done here.
+
+      List<FileWork> blocks = new ArrayList<>();
+      for (DummyColumnsReader reader : readers) {
+        blocks.add(new DummyFileWork(reader.filePath()));
       }
-      return readerIter.next();
+      builder.setConfig(new Configuration());
+      builder.setFiles(blocks);
+      builder.setReaderFactory(new MockFileReaderFactory(readers));
+      return new ColumnsScanFramework(builder);
     }
   }
 
@@ -134,10 +163,11 @@ public class TestColumnsArrayFramework extends SubOperatorTest {
 
     // Create the scan operator
 
-    ColumnsScanOpFixture scanFixture = new ColumnsScanOpFixture();
-    scanFixture.projectAll();
-    scanFixture.addReader(reader);
-    ScanOperatorExec scan = scanFixture.build();
+    ColumnsScanFixtureBuilder builder = new ColumnsScanFixtureBuilder();
+    builder.projectAll();
+    builder.addReader(reader);
+    ScanFixture scanFixture = builder.build();
+    ScanOperatorExec scan = scanFixture.scanOp;
 
     // Start the one and only reader, and check the columns
     // schema info.
@@ -166,10 +196,11 @@ public class TestColumnsArrayFramework extends SubOperatorTest {
 
     // Create the scan operator
 
-    ColumnsScanOpFixture scanFixture = new ColumnsScanOpFixture();
-    scanFixture.setProjection(RowSetTestUtils.projectList(ColumnsArrayManager.COLUMNS_COL));
-    scanFixture.addReader(reader);
-    ScanOperatorExec scan = scanFixture.build();
+    ColumnsScanFixtureBuilder builder = new ColumnsScanFixtureBuilder();
+    builder.setProjection(RowSetTestUtils.projectList(ColumnsArrayManager.COLUMNS_COL));
+    builder.addReader(reader);
+    ScanFixture scanFixture = builder.build();
+    ScanOperatorExec scan = scanFixture.scanOp;
 
     // Start the one and only reader, and check the columns
     // schema info.
@@ -198,12 +229,13 @@ public class TestColumnsArrayFramework extends SubOperatorTest {
 
     // Create the scan operator
 
-    ColumnsScanOpFixture scanFixture = new ColumnsScanOpFixture();
-    scanFixture.setProjection(Lists.newArrayList(
+    ColumnsScanFixtureBuilder builder = new ColumnsScanFixtureBuilder();
+    builder.setProjection(Lists.newArrayList(
         SchemaPath.parseFromString(ColumnsArrayManager.COLUMNS_COL + "[1]"),
         SchemaPath.parseFromString(ColumnsArrayManager.COLUMNS_COL + "[3]")));
-    scanFixture.addReader(reader);
-    ScanOperatorExec scan = scanFixture.build();
+    builder.addReader(reader);
+    ScanFixture scanFixture = builder.build();
+    ScanOperatorExec scan = scanFixture.scanOp;
 
     // Start the one and only reader, and check the columns
     // schema info.
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestColumnsArrayParser.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestColumnsArrayParser.java
index 2a5b00e..4a49033 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestColumnsArrayParser.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestColumnsArrayParser.java
@@ -31,6 +31,7 @@ import org.apache.drill.exec.physical.impl.scan.columns.ColumnsArrayParser;
 import org.apache.drill.exec.physical.impl.scan.columns.UnresolvedColumnsArrayColumn;
 import org.apache.drill.exec.physical.impl.scan.file.FileMetadataColumn;
 import org.apache.drill.exec.physical.impl.scan.file.FileMetadataManager;
+import org.apache.drill.exec.physical.impl.scan.file.FileMetadataManager.FileMetadataOptions;
 import org.apache.drill.exec.physical.impl.scan.project.ScanLevelProjection;
 import org.apache.drill.exec.physical.rowSet.impl.RowSetTestUtils;
 import org.apache.drill.test.SubOperatorTest;
@@ -62,7 +63,7 @@ public class TestColumnsArrayParser extends SubOperatorTest {
 
     // Verify column type
 
-    assertEquals(UnresolvedColumnsArrayColumn.ID, scanProj.columns().get(0).nodeType());
+    assertTrue(scanProj.columns().get(0) instanceof UnresolvedColumnsArrayColumn);
   }
 
   @Test
@@ -79,7 +80,7 @@ public class TestColumnsArrayParser extends SubOperatorTest {
 
     // Verify column type
 
-    assertEquals(UnresolvedColumnsArrayColumn.ID, scanProj.columns().get(0).nodeType());
+    assertTrue(scanProj.columns().get(0) instanceof UnresolvedColumnsArrayColumn);
   }
 
   @Test
@@ -96,7 +97,7 @@ public class TestColumnsArrayParser extends SubOperatorTest {
 
     // Verify column type
 
-    assertEquals(UnresolvedColumnsArrayColumn.ID, scanProj.columns().get(0).nodeType());
+    assertTrue(scanProj.columns().get(0) instanceof UnresolvedColumnsArrayColumn);
   }
 
   @Test
@@ -116,7 +117,7 @@ public class TestColumnsArrayParser extends SubOperatorTest {
 
     // Verify column type
 
-    assertEquals(UnresolvedColumnsArrayColumn.ID, scanProj.columns().get(0).nodeType());
+    assertTrue(scanProj.columns().get(0) instanceof UnresolvedColumnsArrayColumn);
   }
 
   @Test
@@ -136,7 +137,7 @@ public class TestColumnsArrayParser extends SubOperatorTest {
 
     // Verify column type
 
-    assertEquals(UnresolvedColumnsArrayColumn.ID, scanProj.columns().get(0).nodeType());
+    assertTrue(scanProj.columns().get(0) instanceof UnresolvedColumnsArrayColumn);
     UnresolvedColumnsArrayColumn colsCol = (UnresolvedColumnsArrayColumn) scanProj.columns().get(0);
     boolean indexes[] = colsCol.selectedIndexes();
     assertNotNull(indexes);
@@ -222,6 +223,14 @@ public class TestColumnsArrayParser extends SubOperatorTest {
     }
   }
 
+  private FileMetadataOptions standardOptions(Path filePath) {
+    FileMetadataOptions options = new FileMetadataOptions();
+    options.useLegacyWildcardExpansion(false); // Don't expand partition columns for wildcard
+    options.setSelectionRoot(new Path("hdfs:///w"));
+    options.setFiles(Lists.newArrayList(filePath));
+    return options;
+  }
+
   /**
    * The `columns` column is special: can't be used with other column names.
    * Make sure that the rule <i>does not</i> apply to implicit columns.
@@ -232,11 +241,7 @@ public class TestColumnsArrayParser extends SubOperatorTest {
     Path filePath = new Path("hdfs:///w/x/y/z.csv");
     FileMetadataManager metadataManager = new FileMetadataManager(
         fixture.getOptionManager(),
-        false, // Don't expand partition columns for wildcard
-        false, // N/A
-        new Path("hdfs:///w"),
-        FileMetadataManager.AUTO_PARTITION_DEPTH,
-        Lists.newArrayList(filePath));
+        standardOptions(filePath));
 
     ScanLevelProjection scanProj = new ScanLevelProjection(
         RowSetTestUtils.projectList(ScanTestUtils.FILE_NAME_COL,
@@ -255,9 +260,9 @@ public class TestColumnsArrayParser extends SubOperatorTest {
 
     // Verify column type
 
-    assertEquals(FileMetadataColumn.ID, scanProj.columns().get(0).nodeType());
-    assertEquals(UnresolvedColumnsArrayColumn.ID, scanProj.columns().get(1).nodeType());
-    assertEquals(FileMetadataColumn.ID, scanProj.columns().get(2).nodeType());
+    assertTrue(scanProj.columns().get(0) instanceof FileMetadataColumn);
+    assertTrue(scanProj.columns().get(1) instanceof UnresolvedColumnsArrayColumn);
+    assertTrue(scanProj.columns().get(2) instanceof FileMetadataColumn);
   }
 
   /**
@@ -286,7 +291,7 @@ public class TestColumnsArrayParser extends SubOperatorTest {
 
     // Verify column type
 
-    assertEquals(UnresolvedColumnsArrayColumn.ID, scanProj.columns().get(0).nodeType());
+    assertTrue(scanProj.columns().get(0) instanceof UnresolvedColumnsArrayColumn);
   }
 
   @Test
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestFileMetadataColumnParser.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestFileMetadataColumnParser.java
index bbc5e19..f2b6c3b 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestFileMetadataColumnParser.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestFileMetadataColumnParser.java
@@ -27,9 +27,11 @@ import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.physical.impl.scan.file.FileMetadataColumn;
 import org.apache.drill.exec.physical.impl.scan.file.FileMetadataManager;
 import org.apache.drill.exec.physical.impl.scan.file.PartitionColumn;
+import org.apache.drill.exec.physical.impl.scan.file.FileMetadataManager.FileMetadataOptions;
 import org.apache.drill.exec.physical.impl.scan.project.ColumnProjection;
 import org.apache.drill.exec.physical.impl.scan.project.ScanLevelProjection;
-import org.apache.drill.exec.physical.impl.scan.project.UnresolvedColumn;
+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.rowSet.impl.RowSetTestUtils;
 import org.apache.drill.test.SubOperatorTest;
 import org.apache.hadoop.fs.Path;
@@ -40,16 +42,24 @@ import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
 @Category(RowSetTests.class)
 public class TestFileMetadataColumnParser extends SubOperatorTest {
 
+  private FileMetadataOptions standardOptions(Path filePath) {
+    return standardOptions(Lists.newArrayList(filePath));
+  }
+
+  private FileMetadataOptions standardOptions(List<Path> files) {
+    FileMetadataOptions options = new FileMetadataOptions();
+    options.useLegacyWildcardExpansion(false); // Don't expand partition columns for wildcard
+    options.setSelectionRoot(new Path("hdfs:///w"));
+    options.setFiles(files);
+    return options;
+  }
+
   @Test
   public void testBasics() {
     Path filePath = new Path("hdfs:///w/x/y/z.csv");
     FileMetadataManager metadataManager = new FileMetadataManager(
         fixture.getOptionManager(),
-        false, // Don't expand partiton columns for wildcard
-        false, // N/A
-        new Path("hdfs:///w"),
-        FileMetadataManager.AUTO_PARTITION_DEPTH,
-        Lists.newArrayList(filePath));
+        standardOptions(filePath));
 
     // Simulate SELECT a, b, c ...
 
@@ -73,11 +83,7 @@ public class TestFileMetadataColumnParser extends SubOperatorTest {
     Path filePath = new Path("hdfs:///w/x/y/z.csv");
     FileMetadataManager metadataManager = new FileMetadataManager(
         fixture.getOptionManager(),
-        false, // Don't expand partition columns for wildcard
-        false, // N/A
-        new Path("hdfs:///w"),
-        FileMetadataManager.AUTO_PARTITION_DEPTH,
-        Lists.newArrayList(filePath));
+        standardOptions(filePath));
 
     // Simulate SELECT a, fqn, filEPath, filename, suffix ...
 
@@ -102,11 +108,11 @@ public class TestFileMetadataColumnParser extends SubOperatorTest {
 
     // Verify column type
 
-    assertEquals(UnresolvedColumn.UNRESOLVED, scanProj.columns().get(0).nodeType());
-    assertEquals(FileMetadataColumn.ID, scanProj.columns().get(1).nodeType());
-    assertEquals(FileMetadataColumn.ID, scanProj.columns().get(2).nodeType());
-    assertEquals(FileMetadataColumn.ID, scanProj.columns().get(3).nodeType());
-    assertEquals(FileMetadataColumn.ID, scanProj.columns().get(4).nodeType());
+    assertTrue(scanProj.columns().get(0) instanceof UnresolvedColumn);
+    assertTrue(scanProj.columns().get(1) instanceof FileMetadataColumn);
+    assertTrue(scanProj.columns().get(2) instanceof FileMetadataColumn);
+    assertTrue(scanProj.columns().get(3) instanceof FileMetadataColumn);
+    assertTrue(scanProj.columns().get(4) instanceof FileMetadataColumn);
 
     assertTrue(metadataManager.hasImplicitCols());
   }
@@ -120,11 +126,7 @@ public class TestFileMetadataColumnParser extends SubOperatorTest {
     Path filePath = new Path("hdfs:///w/x/y/z.csv");
     FileMetadataManager metadataManager = new FileMetadataManager(
         fixture.getOptionManager(),
-        false, // Don't expand partition columns for wildcard
-        false, // N/A
-        new Path("hdfs:///w"),
-        FileMetadataManager.AUTO_PARTITION_DEPTH,
-        Lists.newArrayList(filePath));
+        standardOptions(filePath));
 
     String dir0 = ScanTestUtils.partitionColName(0);
     // Sic: case insensitivity, but name in project list
@@ -143,7 +145,7 @@ public class TestFileMetadataColumnParser extends SubOperatorTest {
 
     // Verify column type
 
-    assertEquals(PartitionColumn.ID, scanProj.columns().get(0).nodeType());
+    assertTrue(scanProj.columns().get(0) instanceof PartitionColumn);
   }
 
   /**
@@ -155,11 +157,7 @@ public class TestFileMetadataColumnParser extends SubOperatorTest {
     Path filePath = new Path("hdfs:///w/x/y/z.csv");
     FileMetadataManager metadataManager = new FileMetadataManager(
         fixture.getOptionManager(),
-        false, // Don't expand partition columns for wildcard
-        false, // N/A
-        new Path("hdfs:///w"),
-        FileMetadataManager.AUTO_PARTITION_DEPTH,
-        Lists.newArrayList(filePath));
+        standardOptions(filePath));
 
     ScanLevelProjection scanProj = new ScanLevelProjection(
         RowSetTestUtils.projectAll(),
@@ -167,7 +165,7 @@ public class TestFileMetadataColumnParser extends SubOperatorTest {
 
     List<ColumnProjection> cols = scanProj.columns();
     assertEquals(1, cols.size());
-    assertEquals(UnresolvedColumn.WILDCARD, cols.get(0).nodeType());
+    assertTrue(scanProj.columns().get(0) instanceof UnresolvedWildcardColumn);
   }
 
   /**
@@ -178,13 +176,14 @@ public class TestFileMetadataColumnParser extends SubOperatorTest {
   @Test
   public void testLegacyWildcard() {
     Path filePath = new Path("hdfs:///w/x/y/z.csv");
+    FileMetadataOptions options = standardOptions(filePath);
+    options.useLegacyWildcardExpansion(true);
+    options.useLegacyExpansionLocation(true);
+    // Max partition depth is 3, though this "scan" sees only 2
+    options.setPartitionDepth(3);
     FileMetadataManager metadataManager = new FileMetadataManager(
         fixture.getOptionManager(),
-        true, // Use legacy wildcard expansion
-        true, // Put partitions at end
-        new Path("hdfs:///w"),
-        3, // Max partition depth is 3, though this "scan" sees only 2
-        Lists.newArrayList(filePath));
+        options);
 
     ScanLevelProjection scanProj = new ScanLevelProjection(
         RowSetTestUtils.projectAll(),
@@ -192,30 +191,29 @@ public class TestFileMetadataColumnParser extends SubOperatorTest {
 
     List<ColumnProjection> cols = scanProj.columns();
     assertEquals(4, cols.size());
-    assertEquals(UnresolvedColumn.WILDCARD, cols.get(0).nodeType());
-    assertEquals(PartitionColumn.ID, cols.get(1).nodeType());
+    assertTrue(scanProj.columns().get(0) instanceof UnresolvedWildcardColumn);
+    assertTrue(scanProj.columns().get(1) instanceof PartitionColumn);
     assertEquals(0, ((PartitionColumn) cols.get(1)).partition());
-    assertEquals(PartitionColumn.ID, cols.get(2).nodeType());
+    assertTrue(scanProj.columns().get(2) instanceof PartitionColumn);
     assertEquals(1, ((PartitionColumn) cols.get(2)).partition());
-    assertEquals(PartitionColumn.ID, cols.get(3).nodeType());
+    assertTrue(scanProj.columns().get(3) instanceof PartitionColumn);
     assertEquals(2, ((PartitionColumn) cols.get(3)).partition());
   }
 
   /**
-   * Combine wildcard and file metadata columms. The wildcard expands
+   * Combine wildcard and file metadata columns. The wildcard expands
    * table columns but not metadata columns.
    */
 
   @Test
   public void testLegacyWildcardAndFileMetadata() {
     Path filePath = new Path("hdfs:///w/x/y/z.csv");
+    FileMetadataOptions options = standardOptions(filePath);
+    options.useLegacyWildcardExpansion(true);
+    options.useLegacyExpansionLocation(false);
     FileMetadataManager metadataManager = new FileMetadataManager(
         fixture.getOptionManager(),
-        true, // Use legacy wildcard expansion
-        false, // Put partitions at end
-        new Path("hdfs:///w"),
-        FileMetadataManager.AUTO_PARTITION_DEPTH,
-        Lists.newArrayList(filePath));
+        options);
 
     ScanLevelProjection scanProj = new ScanLevelProjection(
         RowSetTestUtils.projectList(
@@ -226,11 +224,11 @@ public class TestFileMetadataColumnParser extends SubOperatorTest {
 
     List<ColumnProjection> cols = scanProj.columns();
     assertEquals(5, cols.size());
-    assertEquals(UnresolvedColumn.WILDCARD, cols.get(0).nodeType());
-    assertEquals(FileMetadataColumn.ID, cols.get(1).nodeType());
-    assertEquals(FileMetadataColumn.ID, cols.get(2).nodeType());
-    assertEquals(PartitionColumn.ID, cols.get(3).nodeType());
-    assertEquals(PartitionColumn.ID, cols.get(4).nodeType());
+    assertTrue(scanProj.columns().get(0) instanceof UnresolvedWildcardColumn);
+    assertTrue(scanProj.columns().get(1) instanceof FileMetadataColumn);
+    assertTrue(scanProj.columns().get(2) instanceof FileMetadataColumn);
+    assertTrue(scanProj.columns().get(3) instanceof PartitionColumn);
+    assertTrue(scanProj.columns().get(4) instanceof PartitionColumn);
   }
 
   /**
@@ -241,13 +239,12 @@ public class TestFileMetadataColumnParser extends SubOperatorTest {
   @Test
   public void testLegacyWildcardAndFileMetadataMixed() {
     Path filePath = new Path("hdfs:///w/x/y/z.csv");
+    FileMetadataOptions options = standardOptions(filePath);
+    options.useLegacyWildcardExpansion(true);
+    options.useLegacyExpansionLocation(false);
     FileMetadataManager metadataManager = new FileMetadataManager(
         fixture.getOptionManager(),
-        true, // Use legacy wildcard expansion
-        false, // Put partitions at end
-        new Path("hdfs:///w"),
-        FileMetadataManager.AUTO_PARTITION_DEPTH,
-        Lists.newArrayList(filePath));
+        options);
 
     ScanLevelProjection scanProj = new ScanLevelProjection(
         RowSetTestUtils.projectList(
@@ -258,11 +255,11 @@ public class TestFileMetadataColumnParser extends SubOperatorTest {
 
     List<ColumnProjection> cols = scanProj.columns();
     assertEquals(5, cols.size());
-    assertEquals(FileMetadataColumn.ID, cols.get(0).nodeType());
-    assertEquals(UnresolvedColumn.WILDCARD, cols.get(1).nodeType());
-    assertEquals(FileMetadataColumn.ID, cols.get(2).nodeType());
-    assertEquals(PartitionColumn.ID, cols.get(3).nodeType());
-    assertEquals(PartitionColumn.ID, cols.get(4).nodeType());
+    assertTrue(scanProj.columns().get(0) instanceof FileMetadataColumn);
+    assertTrue(scanProj.columns().get(1) instanceof UnresolvedWildcardColumn);
+    assertTrue(scanProj.columns().get(2) instanceof FileMetadataColumn);
+    assertTrue(scanProj.columns().get(3) instanceof PartitionColumn);
+    assertTrue(scanProj.columns().get(4) instanceof PartitionColumn);
   }
 
   /**
@@ -279,11 +276,7 @@ public class TestFileMetadataColumnParser extends SubOperatorTest {
     Path filePath = new Path("hdfs:///w/x/y/z.csv");
     FileMetadataManager metadataManager = new FileMetadataManager(
         fixture.getOptionManager(),
-        false, // Don't expand partition columns for wildcard
-        false, // N/A
-        new Path("hdfs:///w"),
-        FileMetadataManager.AUTO_PARTITION_DEPTH,
-        Lists.newArrayList(filePath));
+        standardOptions(filePath));
 
     ScanLevelProjection scanProj = new ScanLevelProjection(
         RowSetTestUtils.projectList(SchemaPath.DYNAMIC_STAR,
@@ -292,20 +285,19 @@ public class TestFileMetadataColumnParser extends SubOperatorTest {
 
       List<ColumnProjection> cols = scanProj.columns();
       assertEquals(2, cols.size());
-      assertEquals(UnresolvedColumn.WILDCARD, cols.get(0).nodeType());
-      assertEquals(PartitionColumn.ID, cols.get(1).nodeType());
+      assertTrue(scanProj.columns().get(0) instanceof UnresolvedWildcardColumn);
+      assertTrue(scanProj.columns().get(1) instanceof PartitionColumn);
   }
 
   @Test
   public void testLegacyWildcardAndPartition() {
     Path filePath = new Path("hdfs:///w/x/y/z.csv");
+    FileMetadataOptions options = standardOptions(filePath);
+    options.useLegacyWildcardExpansion(true);
+    options.useLegacyExpansionLocation(true);
     FileMetadataManager metadataManager = new FileMetadataManager(
         fixture.getOptionManager(),
-        true, // Use legacy wildcard expansion
-        true, // Put partitions at end
-        new Path("hdfs:///w"),
-        FileMetadataManager.AUTO_PARTITION_DEPTH,
-        Lists.newArrayList(filePath));
+        options);
 
     ScanLevelProjection scanProj = new ScanLevelProjection(
         RowSetTestUtils.projectList(SchemaPath.DYNAMIC_STAR,
@@ -314,25 +306,24 @@ public class TestFileMetadataColumnParser extends SubOperatorTest {
 
       List<ColumnProjection> cols = scanProj.columns();
       assertEquals(4, cols.size());
-      assertEquals(UnresolvedColumn.WILDCARD, cols.get(0).nodeType());
-      assertEquals(PartitionColumn.ID, cols.get(1).nodeType());
+      assertTrue(scanProj.columns().get(0) instanceof UnresolvedWildcardColumn);
+      assertTrue(scanProj.columns().get(1) instanceof PartitionColumn);
       assertEquals(0, ((PartitionColumn) cols.get(1)).partition());
-      assertEquals(PartitionColumn.ID, cols.get(2).nodeType());
+      assertTrue(scanProj.columns().get(2) instanceof PartitionColumn);
       assertEquals(1, ((PartitionColumn) cols.get(2)).partition());
-      assertEquals(PartitionColumn.ID, cols.get(3).nodeType());
+      assertTrue(scanProj.columns().get(3) instanceof PartitionColumn);
       assertEquals(8, ((PartitionColumn) cols.get(3)).partition());
   }
 
   @Test
   public void testPreferredPartitionExpansion() {
     Path filePath = new Path("hdfs:///w/x/y/z.csv");
+    FileMetadataOptions options = standardOptions(filePath);
+    options.useLegacyWildcardExpansion(true);
+    options.useLegacyExpansionLocation(false);
     FileMetadataManager metadataManager = new FileMetadataManager(
         fixture.getOptionManager(),
-        true, // Use legacy wildcard expansion
-        false, // Put partitions at end
-        new Path("hdfs:///w"),
-        FileMetadataManager.AUTO_PARTITION_DEPTH,
-        Lists.newArrayList(filePath));
+        options);
 
     ScanLevelProjection scanProj = new ScanLevelProjection(
         RowSetTestUtils.projectList(SchemaPath.DYNAMIC_STAR,
@@ -341,12 +332,12 @@ public class TestFileMetadataColumnParser extends SubOperatorTest {
 
       List<ColumnProjection> cols = scanProj.columns();
       assertEquals(4, cols.size());
-      assertEquals(UnresolvedColumn.WILDCARD, cols.get(0).nodeType());
-      assertEquals(PartitionColumn.ID, cols.get(1).nodeType());
+      assertTrue(scanProj.columns().get(0) instanceof UnresolvedWildcardColumn);
+      assertTrue(scanProj.columns().get(1) instanceof PartitionColumn);
       assertEquals(8, ((PartitionColumn) cols.get(1)).partition());
-      assertEquals(PartitionColumn.ID, cols.get(2).nodeType());
+      assertTrue(scanProj.columns().get(2) instanceof PartitionColumn);
       assertEquals(0, ((PartitionColumn) cols.get(2)).partition());
-      assertEquals(PartitionColumn.ID, cols.get(3).nodeType());
+      assertTrue(scanProj.columns().get(3) instanceof PartitionColumn);
       assertEquals(1, ((PartitionColumn) cols.get(3)).partition());
   }
 
@@ -360,13 +351,12 @@ public class TestFileMetadataColumnParser extends SubOperatorTest {
   @Test
   public void testLegacyWildcardAndPartitionWithOverlap() {
     Path filePath = new Path("hdfs:///w/x/y/z.csv");
+    FileMetadataOptions options = standardOptions(filePath);
+    options.useLegacyWildcardExpansion(true);
+    options.useLegacyExpansionLocation(true);
     FileMetadataManager metadataManager = new FileMetadataManager(
         fixture.getOptionManager(),
-        true, // Use legacy wildcard expansion
-        true, // Put partitions at end
-        new Path("hdfs:///w"),
-        FileMetadataManager.AUTO_PARTITION_DEPTH,
-        Lists.newArrayList(filePath));
+        options);
 
     ScanLevelProjection scanProj = new ScanLevelProjection(
         RowSetTestUtils.projectList(SchemaPath.DYNAMIC_STAR,
@@ -375,23 +365,22 @@ public class TestFileMetadataColumnParser extends SubOperatorTest {
 
       List<ColumnProjection> cols = scanProj.columns();
       assertEquals(3, cols.size());
-      assertEquals(UnresolvedColumn.WILDCARD, cols.get(0).nodeType());
-      assertEquals(PartitionColumn.ID, cols.get(1).nodeType());
+      assertTrue(scanProj.columns().get(0) instanceof UnresolvedWildcardColumn);
+      assertTrue(scanProj.columns().get(1) instanceof PartitionColumn);
       assertEquals(0, ((PartitionColumn) cols.get(1)).partition());
-      assertEquals(PartitionColumn.ID, cols.get(2).nodeType());
+      assertTrue(scanProj.columns().get(2) instanceof PartitionColumn);
       assertEquals(1, ((PartitionColumn) cols.get(2)).partition());
   }
 
   @Test
   public void testPreferedWildcardExpansionWithOverlap() {
     Path filePath = new Path("hdfs:///w/x/y/z.csv");
+    FileMetadataOptions options = standardOptions(filePath);
+    options.useLegacyWildcardExpansion(true);
+    options.useLegacyExpansionLocation(false);
     FileMetadataManager metadataManager = new FileMetadataManager(
         fixture.getOptionManager(),
-        true, // Use legacy wildcard expansion
-        false, // Put partitions at end
-        new Path("hdfs:///w"),
-        FileMetadataManager.AUTO_PARTITION_DEPTH,
-        Lists.newArrayList(filePath));
+        options);
 
     ScanLevelProjection scanProj = new ScanLevelProjection(
         RowSetTestUtils.projectList(SchemaPath.DYNAMIC_STAR,
@@ -400,10 +389,10 @@ public class TestFileMetadataColumnParser extends SubOperatorTest {
 
       List<ColumnProjection> cols = scanProj.columns();
       assertEquals(3, cols.size());
-      assertEquals(UnresolvedColumn.WILDCARD, cols.get(0).nodeType());
-      assertEquals(PartitionColumn.ID, cols.get(1).nodeType());
+      assertTrue(scanProj.columns().get(0) instanceof UnresolvedWildcardColumn);
+      assertTrue(scanProj.columns().get(1) instanceof PartitionColumn);
       assertEquals(1, ((PartitionColumn) cols.get(1)).partition());
-      assertEquals(PartitionColumn.ID, cols.get(2).nodeType());
+      assertTrue(scanProj.columns().get(2) instanceof PartitionColumn);
       assertEquals(0, ((PartitionColumn) cols.get(2)).partition());
   }
 
@@ -418,11 +407,7 @@ public class TestFileMetadataColumnParser extends SubOperatorTest {
     Path filePath = new Path("hdfs:///w/x/y/z.csv");
     FileMetadataManager metadataManager = new FileMetadataManager(
         fixture.getOptionManager(),
-        false, // Don't expand partition columns for wildcard
-        false, // N/A
-        new Path("hdfs:///w"),
-        FileMetadataManager.AUTO_PARTITION_DEPTH,
-        Lists.newArrayList(filePath));
+        standardOptions(filePath));
 
     ScanLevelProjection scanProj = new ScanLevelProjection(
         RowSetTestUtils.projectList(
@@ -436,8 +421,8 @@ public class TestFileMetadataColumnParser extends SubOperatorTest {
     List<ColumnProjection> cols = scanProj.columns();
     assertEquals(5, cols.size());
     for (int i = 0; i < 4; i++) {
-      assertEquals(UnresolvedColumn.UNRESOLVED, cols.get(1).nodeType());
+      assertTrue(scanProj.columns().get(i) instanceof UnresolvedColumn);
     }
-    assertEquals(FileMetadataColumn.ID, cols.get(4).nodeType());
+    assertTrue(scanProj.columns().get(4) instanceof FileMetadataColumn);
   }
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestFileMetadataProjection.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestFileMetadataProjection.java
index dae89ab..6ef69cf 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestFileMetadataProjection.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestFileMetadataProjection.java
@@ -34,13 +34,15 @@ import org.apache.drill.exec.physical.impl.scan.file.FileMetadataColumn;
 import org.apache.drill.exec.physical.impl.scan.file.FileMetadataManager;
 import org.apache.drill.exec.physical.impl.scan.file.MetadataColumn;
 import org.apache.drill.exec.physical.impl.scan.file.PartitionColumn;
+import org.apache.drill.exec.physical.impl.scan.file.FileMetadataManager.FileMetadataOptions;
 import org.apache.drill.exec.physical.impl.scan.project.ColumnProjection;
 import org.apache.drill.exec.physical.impl.scan.project.ExplicitSchemaProjection;
 import org.apache.drill.exec.physical.impl.scan.project.NullColumnBuilder;
+import org.apache.drill.exec.physical.impl.scan.project.NullColumnBuilder.NullBuilderBuilder;
 import org.apache.drill.exec.physical.impl.scan.project.ResolvedColumn;
 import org.apache.drill.exec.physical.impl.scan.project.ResolvedTuple.ResolvedRow;
 import org.apache.drill.exec.physical.impl.scan.project.ScanLevelProjection;
-import org.apache.drill.exec.physical.impl.scan.project.UnresolvedColumn;
+import org.apache.drill.exec.physical.impl.scan.project.AbstractUnresolvedColumn.UnresolvedColumn;
 import org.apache.drill.exec.physical.rowSet.impl.RowSetTestUtils;
 import org.apache.drill.exec.record.metadata.SchemaBuilder;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
@@ -154,6 +156,18 @@ public class TestFileMetadataProjection extends SubOperatorTest {
     }
   }
 
+  private FileMetadataOptions standardOptions(Path filePath) {
+    return standardOptions(Lists.newArrayList(filePath));
+  }
+
+  private FileMetadataOptions standardOptions(List<Path> files) {
+    FileMetadataOptions options = new FileMetadataOptions();
+    options.useLegacyWildcardExpansion(false); // Don't expand partition columns for wildcard
+    options.setSelectionRoot(new Path("hdfs:///w"));
+    options.setFiles(files);
+    return options;
+  }
+
   /**
    * Test the file projection planner with metadata.
    */
@@ -163,11 +177,7 @@ public class TestFileMetadataProjection extends SubOperatorTest {
     Path filePath = new Path("hdfs:///w/x/y/z.csv");
     FileMetadataManager metadataManager = new FileMetadataManager(
         fixture.getOptionManager(),
-        false, // Don't expand partiton columns for wildcard
-        false, // N/A
-        new Path("hdfs:///w"),
-        FileMetadataManager.AUTO_PARTITION_DEPTH,
-        Lists.newArrayList(filePath));
+        standardOptions(filePath));
 
     ScanLevelProjection scanProj = new ScanLevelProjection(
         RowSetTestUtils.projectList(
@@ -182,18 +192,18 @@ public class TestFileMetadataProjection extends SubOperatorTest {
     {
       assertTrue(scanProj.columns().get(0) instanceof FileMetadataColumn);
       FileMetadataColumn col0 = (FileMetadataColumn) scanProj.columns().get(0);
-      assertEquals(FileMetadataColumn.ID, col0.nodeType());
+      assertTrue(col0 instanceof FileMetadataColumn);
       assertEquals(ScanTestUtils.FILE_NAME_COL, col0.name());
       assertEquals(MinorType.VARCHAR, col0.schema().getType().getMinorType());
       assertEquals(DataMode.REQUIRED, col0.schema().getType().getMode());
 
       ColumnProjection col1 = scanProj.columns().get(1);
-      assertEquals(UnresolvedColumn.UNRESOLVED, col1.nodeType());
+      assertTrue(col1 instanceof UnresolvedColumn);
       assertEquals("a", col1.name());
 
       assertTrue(scanProj.columns().get(2) instanceof PartitionColumn);
       PartitionColumn col2 = (PartitionColumn) scanProj.columns().get(2);
-      assertEquals(PartitionColumn.ID, col2.nodeType());
+      assertTrue(col2 instanceof PartitionColumn);
       assertEquals(ScanTestUtils.partitionColName(0), col2.name());
       assertEquals(MinorType.VARCHAR, col2.schema().getType().getMinorType());
       assertEquals(DataMode.OPTIONAL, col2.schema().getType().getMode());
@@ -206,7 +216,7 @@ public class TestFileMetadataProjection extends SubOperatorTest {
         .buildSchema();
 
     metadataManager.startFile(filePath);
-    NullColumnBuilder builder = new NullColumnBuilder(null, false);
+    NullColumnBuilder builder = new NullBuilderBuilder().build();
     ResolvedRow rootTuple = new ResolvedRow(builder);
     new ExplicitSchemaProjection(
         scanProj, tableSchema, rootTuple,
@@ -218,7 +228,7 @@ public class TestFileMetadataProjection extends SubOperatorTest {
     {
       assertTrue(columns.get(0) instanceof FileMetadataColumn);
       FileMetadataColumn col0 = (FileMetadataColumn) columns.get(0);
-      assertEquals(FileMetadataColumn.ID, col0.nodeType());
+      assertTrue(col0 instanceof FileMetadataColumn);
       assertEquals(ScanTestUtils.FILE_NAME_COL, col0.name());
       assertEquals("z.csv", col0.value());
       assertEquals(MinorType.VARCHAR, col0.schema().getType().getMinorType());
@@ -229,7 +239,7 @@ public class TestFileMetadataProjection extends SubOperatorTest {
 
       assertTrue(columns.get(2) instanceof PartitionColumn);
       PartitionColumn col2 = (PartitionColumn) columns.get(2);
-      assertEquals(PartitionColumn.ID, col2.nodeType());
+      assertTrue(col2 instanceof PartitionColumn);
       assertEquals(ScanTestUtils.partitionColName(0), col2.name());
       assertEquals("x", col2.value());
       assertEquals(MinorType.VARCHAR, col2.schema().getType().getMinorType());
@@ -252,11 +262,7 @@ public class TestFileMetadataProjection extends SubOperatorTest {
     Path filePath = new Path("hdfs:///w/x/y/z.csv");
     FileMetadataManager metadataManager = new FileMetadataManager(
         fixture.getOptionManager(),
-        false, // Don't expand partition columns for wildcard
-        false, // N/A
-        new Path("hdfs:///w"),
-        FileMetadataManager.AUTO_PARTITION_DEPTH,
-        Lists.newArrayList(filePath));
+        standardOptions(filePath));
 
     ScanLevelProjection scanProj = new ScanLevelProjection(
         RowSetTestUtils.projectList(
@@ -280,7 +286,7 @@ public class TestFileMetadataProjection extends SubOperatorTest {
         .buildSchema();
 
     metadataManager.startFile(filePath);
-    NullColumnBuilder builder = new NullColumnBuilder(null, false);
+    NullColumnBuilder builder = new NullBuilderBuilder().build();
     ResolvedRow rootTuple = new ResolvedRow(builder);
     new ExplicitSchemaProjection(
         scanProj, tableSchema, rootTuple,
@@ -306,11 +312,7 @@ public class TestFileMetadataProjection extends SubOperatorTest {
     Path filePath = new Path("hdfs:///x/0/1/2/3/4/5/6/7/8/9/10/d11/z.csv");
     FileMetadataManager metadataManager = new FileMetadataManager(
         fixture.getOptionManager(),
-        false, // Don't expand partition columns for wildcard
-        false, // N/A
-        new Path("hdfs:///x"),
-        FileMetadataManager.AUTO_PARTITION_DEPTH,
-        Lists.newArrayList(filePath));
+        standardOptions(filePath));
 
     ScanLevelProjection scanProj = new ScanLevelProjection(
         RowSetTestUtils.projectList("dir11"),
@@ -321,7 +323,7 @@ public class TestFileMetadataProjection extends SubOperatorTest {
         .buildSchema();
 
     metadataManager.startFile(filePath);
-    NullColumnBuilder builder = new NullColumnBuilder(null, false);
+    NullColumnBuilder builder = new NullBuilderBuilder().build();
     ResolvedRow rootTuple = new ResolvedRow(builder);
     new ExplicitSchemaProjection(
         scanProj, tableSchema, rootTuple,
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestFileScanFramework.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestFileScanFramework.java
index 7b9fbfb..29ee78c 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestFileScanFramework.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestFileScanFramework.java
@@ -26,15 +26,17 @@ import java.util.Iterator;
 import java.util.List;
 
 import org.apache.drill.categories.RowSetTests;
-import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.common.types.TypeProtos.MinorType;
-import org.apache.drill.exec.physical.impl.scan.TestScanOperatorExec.AbstractScanOpFixture;
-import org.apache.drill.exec.physical.impl.scan.file.BaseFileScanFramework;
-import org.apache.drill.exec.physical.impl.scan.file.BaseFileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.ScanTestUtils.ScanFixture;
+import org.apache.drill.exec.physical.impl.scan.ScanTestUtils.ScanFixtureBuilder;
 import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
 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.physical.impl.scan.framework.ManagedScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedScanFramework.ScanFrameworkBuilder;
 import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
 import org.apache.drill.exec.physical.rowSet.RowSetLoader;
 import org.apache.drill.exec.record.BatchSchema;
@@ -42,11 +44,10 @@ import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.metadata.ColumnBuilder;
 import org.apache.drill.exec.record.metadata.SchemaBuilder;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
-import org.apache.drill.exec.store.dfs.DrillFileSystem;
 import org.apache.drill.exec.store.dfs.easy.FileWork;
 import org.apache.drill.test.SubOperatorTest;
 import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
-import org.apache.drill.test.rowSet.RowSetComparison;
+import org.apache.drill.test.rowSet.RowSetUtilities;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.FileSplit;
@@ -72,8 +73,7 @@ public class TestFileScanFramework extends SubOperatorTest {
   private static final String MOCK_DIR1 = "y";
 
   /**
-   * For schema-based testing, we only need the file path
-   * from the file work
+   * For schema-based testing, we only need the file path from the file work.
    */
 
   public static class DummyFileWork implements FileWork {
@@ -94,64 +94,73 @@ public class TestFileScanFramework extends SubOperatorTest {
     public long getLength() { return 0; }
   }
 
+  private interface MockFileReader extends ManagedReader<FileSchemaNegotiator> {
+    Path filePath();
+  }
+
   /**
-   * Fixture class to assemble all the knick-knacks that make up a
-   * file scan framework. The parts have a form unique to testing
-   * since we are not actually doing real scans.
+   * Mock file reader that returns readers already created for specific
+   * test cases. Verifies that the readers match the file splits
+   * (which were obtained from the readers.)
+   * <p>
+   * This is not a good example of a real file reader factory, it does,
+   * however, illustrate a design goal to allow a variety of implementations
+   * through composition.
    */
+  public static class MockFileReaderFactory extends FileReaderFactory {
+    public Iterator<MockFileReader> readerIter;
 
-  public abstract static class BaseFileScanOpFixture extends AbstractScanOpFixture {
-
-    protected Path selectionRoot = MOCK_ROOT_PATH;
-    protected int partitionDepth = 3;
-    protected List<FileWork> files = new ArrayList<>();
-    protected Configuration fsConfig = new Configuration();
-
-    public ScanOperatorExec build() {
-      BaseFileScanFramework<?> framework = buildFramework();
-      configure(framework);
-      configureFileScan(framework);
-      return buildScanOp(framework);
+    public MockFileReaderFactory(List<MockFileReader> readers) {
+      readerIter = readers.iterator();
     }
 
-    protected abstract BaseFileScanFramework<?> buildFramework();
-
-    private void configureFileScan(BaseFileScanFramework<?> framework) {
-      framework.setSelectionRoot(selectionRoot, partitionDepth);
+    @Override
+    public ManagedReader<? extends FileSchemaNegotiator> newReader(
+        FileSplit split) {
+      MockFileReader reader = readerIter.next();
+      assert reader != null;
+      assert split.getPath().equals(reader.filePath());
+      return reader;
     }
   }
 
-  public static class FileScanOpFixture extends BaseFileScanOpFixture implements FileReaderFactory {
+  public static class FileScanFixtureBuilder extends ScanFixtureBuilder {
 
-    protected final List<MockFileReader> readers = new ArrayList<>();
-    protected Iterator<MockFileReader> readerIter;
+    public FileScanBuilder builder = new FileScanBuilder();
+    public List<MockFileReader> readers = new ArrayList<>();
 
-    public void addReader(MockFileReader reader) {
-      readers.add(reader);
-      files.add(new DummyFileWork(reader.filePath()));
+    public FileScanFixtureBuilder() {
+      super(fixture);
+      builder.metadataOptions().setSelectionRoot(MOCK_ROOT_PATH);
+      builder.metadataOptions().setPartitionDepth(3);
     }
 
     @Override
-    protected BaseFileScanFramework<?> buildFramework() {
-      readerIter = readers.iterator();
-      return new FileScanFramework(projection, files, fsConfig, this);
+    public ScanFrameworkBuilder builder() { return builder; }
+
+    public void addReader(MockFileReader reader) {
+      readers.add(reader);
     }
 
     @Override
-    public ManagedReader<FileSchemaNegotiator> makeBatchReader(
-        DrillFileSystem dfs,
-        FileSplit split) throws ExecutionSetupException {
-      if (! readerIter.hasNext()) {
-        return null;
+    protected ManagedScanFramework newFramework() {
+
+      // Bass-ackward construction of the list of files from
+      // a set of text fixture readers. Normal implementations
+      // create readers from file splits, not the other way around
+      // as is done here.
+
+      List<FileWork> blocks = new ArrayList<>();
+      for (MockFileReader reader : readers) {
+        blocks.add(new DummyFileWork(reader.filePath()));
       }
-      return readerIter.next();
+      builder.setConfig(new Configuration());
+      builder.setFiles(blocks);
+      builder.setReaderFactory(new MockFileReaderFactory(readers));
+      return new FileScanFramework(builder);
     }
   }
 
-  private interface MockFileReader extends ManagedReader<FileSchemaNegotiator> {
-    Path filePath();
-  }
-
   /**
    * Base class for the "mock" readers used in this test. The mock readers
    * follow the normal (enhanced) reader API, but instead of actually reading
@@ -284,10 +293,11 @@ public class TestFileScanFramework extends SubOperatorTest {
 
     // Create the scan operator
 
-    FileScanOpFixture scanFixture = new FileScanOpFixture();
-    scanFixture.projectAllWithMetadata(2);
-    scanFixture.addReader(reader);
-    ScanOperatorExec scan = scanFixture.build();
+    FileScanFixtureBuilder builder = new FileScanFixtureBuilder();
+    builder.projectAllWithMetadata(2);
+    builder.addReader(reader);
+    ScanFixture scanFixture = builder.build();
+    ScanOperatorExec scan = scanFixture.scanOp;
 
     // Standard startup
 
@@ -318,13 +328,13 @@ public class TestFileScanFramework extends SubOperatorTest {
         .addRow(30, "fred", MOCK_FILE_FQN, MOCK_FILE_PATH, MOCK_FILE_NAME, MOCK_SUFFIX, MOCK_DIR0, MOCK_DIR1, null)
         .addRow(40, "wilma", MOCK_FILE_FQN, MOCK_FILE_PATH, MOCK_FILE_NAME, MOCK_SUFFIX, MOCK_DIR0, MOCK_DIR1, null)
         .build();
-    RowSetComparison verifier = new RowSetComparison(expected);
     assertEquals(expected.batchSchema(), scan.batchAccessor().getSchema());
 
     // Next call, return with data.
 
     assertTrue(scan.next());
-    verifier.verifyAndClearAll(fixture.wrap(scan.batchAccessor().getOutgoingContainer()));
+    RowSetUtilities.verify(expected,
+        fixture.wrap(scan.batchAccessor().getOutgoingContainer()));
 
     // EOF
 
@@ -349,10 +359,11 @@ public class TestFileScanFramework extends SubOperatorTest {
 
     // Select table and implicit columns.
 
-    FileScanOpFixture scanFixture = new FileScanOpFixture();
-    scanFixture.setProjection(new String[] {"a", "b", "filename", "suffix"});
-    scanFixture.addReader(reader);
-    ScanOperatorExec scan = scanFixture.build();
+    FileScanFixtureBuilder builder = new FileScanFixtureBuilder();
+    builder.setProjection(new String[] {"a", "b", "filename", "suffix"});
+    builder.addReader(reader);
+    ScanFixture scanFixture = builder.build();
+    ScanOperatorExec scan = scanFixture.scanOp;
 
     // Expect data and implicit columns
 
@@ -366,7 +377,6 @@ public class TestFileScanFramework extends SubOperatorTest {
         .addRow(10, "fred", MOCK_FILE_NAME, MOCK_SUFFIX)
         .addRow(20, "wilma", MOCK_FILE_NAME, MOCK_SUFFIX)
         .build();
-    RowSetComparison verifier = new RowSetComparison(expected);
 
     // Schema should include implicit columns.
 
@@ -377,7 +387,8 @@ public class TestFileScanFramework extends SubOperatorTest {
     // Read one batch, should contain implicit columns
 
     assertTrue(scan.next());
-    verifier.verifyAndClearAll(fixture.wrap(scan.batchAccessor().getOutgoingContainer()));
+    RowSetUtilities.verify(expected,
+        fixture.wrap(scan.batchAccessor().getOutgoingContainer()));
 
     // EOF
 
@@ -402,10 +413,11 @@ public class TestFileScanFramework extends SubOperatorTest {
 
     // Select table and implicit columns.
 
-    FileScanOpFixture scanFixture = new FileScanOpFixture();
-    scanFixture.setProjection(new String[] {"dir0", "b", "filename", "c", "suffix"});
-    scanFixture.addReader(reader);
-    ScanOperatorExec scan = scanFixture.build();
+    FileScanFixtureBuilder builder = new FileScanFixtureBuilder();
+    builder.setProjection(new String[] {"dir0", "b", "filename", "c", "suffix"});
+    builder.addReader(reader);
+    ScanFixture scanFixture = builder.build();
+    ScanOperatorExec scan = scanFixture.scanOp;
 
     // Expect data and implicit columns
 
@@ -420,7 +432,6 @@ public class TestFileScanFramework extends SubOperatorTest {
         .addRow(MOCK_DIR0, "fred", MOCK_FILE_NAME, null, MOCK_SUFFIX)
         .addRow(MOCK_DIR0, "wilma", MOCK_FILE_NAME, null, MOCK_SUFFIX)
         .build();
-    RowSetComparison verifier = new RowSetComparison(expected);
 
     // Schema should include implicit columns.
 
@@ -431,7 +442,8 @@ public class TestFileScanFramework extends SubOperatorTest {
     // Read one batch, should contain implicit columns
 
     assertTrue(scan.next());
-    verifier.verifyAndClearAll(fixture.wrap(scan.batchAccessor().getOutgoingContainer()));
+    RowSetUtilities.verify(expected,
+        fixture.wrap(scan.batchAccessor().getOutgoingContainer()));
 
     // EOF
 
@@ -448,10 +460,11 @@ public class TestFileScanFramework extends SubOperatorTest {
 
     // Select no columns
 
-    FileScanOpFixture scanFixture = new FileScanOpFixture();
-    scanFixture.setProjection(new String[] {});
-    scanFixture.addReader(reader);
-    ScanOperatorExec scan = scanFixture.build();
+    FileScanFixtureBuilder builder = new FileScanFixtureBuilder();
+    builder.setProjection(new String[] {});
+    builder.addReader(reader);
+    ScanFixture scanFixture = builder.build();
+    ScanOperatorExec scan = scanFixture.scanOp;
 
     // Expect data and implicit columns
 
@@ -461,7 +474,6 @@ public class TestFileScanFramework extends SubOperatorTest {
         .addRow()
         .addRow()
         .build();
-    RowSetComparison verifier = new RowSetComparison(expected);
 
     // Schema should include implicit columns.
 
@@ -472,7 +484,8 @@ public class TestFileScanFramework extends SubOperatorTest {
     // Read one batch, should contain implicit columns
 
     assertTrue(scan.next());
-    verifier.verifyAndClearAll(fixture.wrap(scan.batchAccessor().getOutgoingContainer()));
+    RowSetUtilities.verify(expected,
+        fixture.wrap(scan.batchAccessor().getOutgoingContainer()));
 
     // EOF
 
@@ -518,10 +531,11 @@ public class TestFileScanFramework extends SubOperatorTest {
 
     // Select one of the two map columns
 
-    FileScanOpFixture scanFixture = new FileScanOpFixture();
-    scanFixture.setProjection(new String[] {"m1.a"});
-    scanFixture.addReader(reader);
-    ScanOperatorExec scan = scanFixture.build();
+    FileScanFixtureBuilder builder = new FileScanFixtureBuilder();
+    builder.setProjection(new String[] {"m1.a"});
+    builder.addReader(reader);
+    ScanFixture scanFixture = builder.build();
+    ScanOperatorExec scan = scanFixture.scanOp;
 
     // Expect data and implicit columns
 
@@ -539,8 +553,8 @@ public class TestFileScanFramework extends SubOperatorTest {
     scan.batchAccessor().release();
 
     assertTrue(scan.next());
-    new RowSetComparison(expected)
-         .verifyAndClearAll(fixture.wrap(scan.batchAccessor().getOutgoingContainer()));
+    RowSetUtilities.verify(expected,
+         fixture.wrap(scan.batchAccessor().getOutgoingContainer()));
 
     // EOF
 
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOperExecBasics.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOperExecBasics.java
new file mode 100644
index 0000000..dd07d5a
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOperExecBasics.java
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.scan;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.apache.drill.categories.RowSetTests;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.physical.impl.scan.ScanTestUtils.ScanFixture;
+import org.apache.drill.exec.physical.impl.scan.framework.SchemaNegotiator;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Tests the basics of the scan operator protocol: error conditions,
+ * etc.
+ */
+@Category(RowSetTests.class)
+public class TestScanOperExecBasics extends BaseScanOperatorExecTest {
+
+  /**
+   * Pathological case that a scan operator is provided no readers.
+   * It will throw a user exception because the downstream operators
+   * can't handle this case so we choose to stop the show early to
+   * avoid getting into a strange state.
+   */
+  @Test
+  public void testNoReader() {
+
+    // Create the scan operator
+
+    ScanFixture scanFixture = simpleFixture();
+    ScanOperatorExec scan = scanFixture.scanOp;
+
+    try {
+      scan.buildSchema();
+    } catch (UserException e) {
+
+      // Expected
+
+      assertTrue(e.getCause() instanceof ExecutionSetupException);
+    }
+
+    // Must close the DAG (context and scan operator) even on failures
+
+    scanFixture.close();
+  }
+
+  public final String ERROR_MSG = "My Bad!";
+
+  @Test
+  public void testExceptionOnOpen() {
+
+    // Reader which fails on open with a known error message
+    // using an exception other than UserException.
+
+    MockEarlySchemaReader reader = new MockEarlySchemaReader() {
+      @Override
+      public boolean open(SchemaNegotiator schemaNegotiator) {
+        openCalled = true;
+        throw new IllegalStateException(ERROR_MSG);
+      }
+
+    };
+    reader.batchLimit = 0;
+
+    ScanFixture scanFixture = simpleFixture(reader);
+    ScanOperatorExec scan = scanFixture.scanOp;
+
+    try {
+      scan.buildSchema();
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains(ERROR_MSG));
+      assertTrue(e.getCause() instanceof IllegalStateException);
+    }
+    assertTrue(reader.openCalled);
+
+    assertEquals(0, scan.batchAccessor().getRowCount());
+    scanFixture.close();
+    assertTrue(reader.closeCalled);
+  }
+
+  @Test
+  public void testUserExceptionOnOpen() {
+
+    // Reader which fails on open with a known error message
+    // using a UserException.
+
+    MockEarlySchemaReader reader = new MockEarlySchemaReader() {
+      @Override
+      public boolean open(SchemaNegotiator schemaNegotiator) {
+        openCalled = true;
+        throw UserException.dataReadError()
+            .addContext(ERROR_MSG)
+            .build(logger);
+      }
+
+    };
+    reader.batchLimit = 2;
+
+    ScanFixture scanFixture = simpleFixture(reader);
+    ScanOperatorExec scan = scanFixture.scanOp;
+
+    try {
+      scan.buildSchema();
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains(ERROR_MSG));
+      assertNull(e.getCause());
+    }
+    assertTrue(reader.openCalled);
+
+    assertEquals(0, scan.batchAccessor().getRowCount());
+    scanFixture.close();
+    assertTrue(reader.closeCalled);
+  }
+
+  @Test
+  public void testExceptionOnFirstNext() {
+    MockEarlySchemaReader reader = new MockEarlySchemaReader() {
+      @Override
+      public boolean next() {
+        super.next(); // Load some data
+        throw new IllegalStateException(ERROR_MSG);
+      }
+    };
+    reader.batchLimit = 2;
+
+    ScanFixture scanFixture = simpleFixture(reader);
+    ScanOperatorExec scan = scanFixture.scanOp;
+
+    assertTrue(scan.buildSchema());
+
+    try {
+      scan.next();
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains(ERROR_MSG));
+      assertTrue(e.getCause() instanceof IllegalStateException);
+    }
+    assertTrue(reader.openCalled);
+
+    assertEquals(0, scan.batchAccessor().getRowCount());
+    scanFixture.close();
+    assertTrue(reader.closeCalled);
+  }
+
+  @Test
+  public void testUserExceptionOnFirstNext() {
+    MockEarlySchemaReader reader = new MockEarlySchemaReader() {
+      @Override
+      public boolean next() {
+        super.next(); // Load some data
+        throw UserException.dataReadError()
+            .addContext(ERROR_MSG)
+            .build(logger);
+      }
+    };
+    reader.batchLimit = 2;
+
+    ScanFixture scanFixture = simpleFixture(reader);
+    ScanOperatorExec scan = scanFixture.scanOp;
+
+    assertTrue(scan.buildSchema());
+
+    // EOF
+
+    try {
+      scan.next();
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains(ERROR_MSG));
+      assertNull(e.getCause());
+    }
+    assertTrue(reader.openCalled);
+
+    assertEquals(0, scan.batchAccessor().getRowCount());
+    scanFixture.close();
+    assertTrue(reader.closeCalled);
+  }
+
+  /**
+   * Test throwing an exception after the first batch, but while
+   * "reading" the second. Note that the first batch returns data
+   * and is spread over two next() calls, so the error is on the
+   * third call to the scan operator next().
+   */
+
+  @Test
+  public void testExceptionOnSecondNext() {
+    MockEarlySchemaReader reader = new MockEarlySchemaReader() {
+      @Override
+      public boolean next() {
+        if (batchCount == 1) {
+          super.next(); // Load some data
+          throw new IllegalStateException(ERROR_MSG);
+        }
+        return super.next();
+      }
+    };
+    reader.batchLimit = 2;
+
+    ScanFixture scanFixture = simpleFixture(reader);
+    ScanOperatorExec scan = scanFixture.scanOp;
+
+    // Schema
+
+    assertTrue(scan.buildSchema());
+
+    // First batch
+
+    assertTrue(scan.next());
+    scan.batchAccessor().release();
+
+    // Fail
+
+    try {
+      scan.next();
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains(ERROR_MSG));
+      assertTrue(e.getCause() instanceof IllegalStateException);
+    }
+
+    scanFixture.close();
+    assertTrue(reader.closeCalled);
+  }
+
+  @Test
+  public void testUserExceptionOnSecondNext() {
+    MockEarlySchemaReader reader = new MockEarlySchemaReader() {
+      @Override
+      public boolean next() {
+        if (batchCount == 1) {
+          super.next(); // Load some data
+          throw UserException.dataReadError()
+              .addContext(ERROR_MSG)
+              .build(logger);
+        }
+        return super.next();
+      }
+    };
+    reader.batchLimit = 2;
+
+    ScanFixture scanFixture = simpleFixture(reader);
+    ScanOperatorExec scan = scanFixture.scanOp;
+
+    // Schema
+
+    assertTrue(scan.buildSchema());
+
+    // First batch
+
+    assertTrue(scan.next());
+    scan.batchAccessor().release();
+
+    // Fail
+
+    try {
+      scan.next();
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains(ERROR_MSG));
+      assertNull(e.getCause());
+    }
+
+    scanFixture.close();
+    assertTrue(reader.closeCalled);
+  }
+
+  @Test
+  public void testExceptionOnClose() {
+    MockEarlySchemaReader reader1 = new MockEarlySchemaReader() {
+      @Override
+      public void close() {
+        super.close();
+        throw new IllegalStateException(ERROR_MSG);
+       }
+    };
+    reader1.batchLimit = 2;
+
+    MockEarlySchemaReader reader2 = new MockEarlySchemaReader();
+    reader2.batchLimit = 2;
+
+    ScanFixture scanFixture = simpleFixture(reader1, reader2);
+    ScanOperatorExec scan = scanFixture.scanOp;
+
+    assertTrue(scan.buildSchema());
+
+    assertTrue(scan.next());
+    scan.batchAccessor().release();
+
+    assertTrue(scan.next());
+    scan.batchAccessor().release();
+
+    // Fail on close of first reader
+
+    try {
+      scan.next();
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains(ERROR_MSG));
+      assertTrue(e.getCause() instanceof IllegalStateException);
+    }
+    assertTrue(reader1.closeCalled);
+    assertFalse(reader2.openCalled);
+
+    scanFixture.close();
+  }
+
+  @Test
+  public void testUserExceptionOnClose() {
+    MockEarlySchemaReader reader1 = new MockEarlySchemaReader() {
+      @Override
+      public void close() {
+        super.close();
+        throw UserException.dataReadError()
+            .addContext(ERROR_MSG)
+            .build(logger);
+       }
+    };
+    reader1.batchLimit = 2;
+
+    MockEarlySchemaReader reader2 = new MockEarlySchemaReader();
+    reader2.batchLimit = 2;
+
+    ScanFixture scanFixture = simpleFixture(reader1, reader2);
+    ScanOperatorExec scan = scanFixture.scanOp;
+
+    assertTrue(scan.buildSchema());
+
+    assertTrue(scan.next());
+    scan.batchAccessor().release();
+
+    assertTrue(scan.next());
+    scan.batchAccessor().release();
+
+    // Fail on close of first reader
+
+    try {
+      scan.next();
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains(ERROR_MSG));
+      assertNull(e.getCause());
+    }
+    assertTrue(reader1.closeCalled);
+    assertFalse(reader2.openCalled);
+
+    scanFixture.close();
+  }
+
+  /**
+   * Test multiple readers, all EOF on first batch.
+   */
+
+  @Test
+  public void testMultiEOFOnFirstBatch() {
+    MockEarlySchemaReader reader1 = new MockEarlySchemaReader();
+    reader1.batchLimit = 0;
+    MockEarlySchemaReader reader2 = new MockEarlySchemaReader();
+    reader2.batchLimit = 0;
+
+    ScanFixture scanFixture = simpleFixture(reader1, reader2);
+    ScanOperatorExec scan = scanFixture.scanOp;
+
+    // EOF
+
+    assertTrue(scan.buildSchema());
+    assertFalse(scan.next());
+    assertTrue(reader1.closeCalled);
+    assertTrue(reader2.closeCalled);
+    assertEquals(0, scan.batchAccessor().getRowCount());
+
+    scanFixture.close();
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOperExecEarlySchema.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOperExecEarlySchema.java
new file mode 100644
index 0000000..a7a4f64
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOperExecEarlySchema.java
@@ -0,0 +1,260 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.scan;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.drill.categories.RowSetTests;
+import org.apache.drill.exec.physical.impl.scan.ScanTestUtils.ScanFixture;
+import org.apache.drill.exec.physical.impl.scan.framework.SchemaNegotiator;
+import org.apache.drill.test.rowSet.RowSetComparison;
+import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test "early schema" readers: those that can declare a schema at
+ * open time.
+ */
+@Category(RowSetTests.class)
+public class TestScanOperExecEarlySchema extends BaseScanOperatorExecTest {
+
+  /**
+   * Mock reader that returns no schema and no records.
+   */
+
+  private static class MockNullEarlySchemaReader extends BaseMockBatchReader {
+
+    @Override
+    public boolean open(SchemaNegotiator schemaNegotiator) {
+      openCalled = true;
+      return false;
+    }
+
+    @Override
+    public boolean next() {
+      return false;
+    }
+  }
+
+  @Test
+  public void testEarlySchemaLifecycle() {
+
+    // Create a mock reader, return two batches: one schema-only, another with data.
+
+    MockEarlySchemaReader reader = new MockEarlySchemaReader();
+    reader.batchLimit = 1;
+
+    // Create the scan operator
+
+    ScanFixture scanFixture = simpleFixture(reader);
+    ScanOperatorExec scan = scanFixture.scanOp;
+
+    SingleRowSet expected = makeExpected();
+    RowSetComparison verifier = new RowSetComparison(expected);
+
+    // First batch: return schema.
+
+    assertTrue(scan.buildSchema());
+    assertEquals(0, reader.batchCount);
+    assertEquals(expected.batchSchema(), scan.batchAccessor().getSchema());
+    assertEquals(0, scan.batchAccessor().getRowCount());
+
+    // Next call, return with data.
+
+    assertTrue(scan.next());
+    verifier.verifyAndClearAll(fixture.wrap(scan.batchAccessor().getOutgoingContainer()));
+
+    // EOF
+
+    assertFalse(scan.next());
+    assertEquals(0, scan.batchAccessor().getRowCount());
+
+    // Next again: no-op
+
+    assertFalse(scan.next());
+    scanFixture.close();
+
+    // Close again: no-op
+
+    scan.close();
+  }
+
+  private static class MockEarlySchemaReader3 extends MockEarlySchemaReader {
+
+    @Override
+    public boolean next() {
+      if (batchCount >= batchLimit) {
+        return false;
+      }
+      batchCount++;
+
+      makeBatch();
+      return batchCount < batchLimit;
+    }
+  }
+
+  @Test
+  public void testEarlySchemaDataWithEof() {
+
+    // Create a mock reader, return two batches: one schema-only, another with data.
+
+    MockEarlySchemaReader3 reader = new MockEarlySchemaReader3();
+    reader.batchLimit = 1;
+
+    // Create the scan operator
+
+    ScanFixture scanFixture = simpleFixture(reader);
+    ScanOperatorExec scan = scanFixture.scanOp;
+
+    SingleRowSet expected = makeExpected();
+    RowSetComparison verifier = new RowSetComparison(expected);
+
+    // First batch: return schema.
+
+    assertTrue(scan.buildSchema());
+    assertEquals(0, scan.batchAccessor().getRowCount());
+
+    // Next call, return with data.
+
+    assertTrue(scan.next());
+    verifier.verifyAndClearAll(fixture.wrap(scan.batchAccessor().getOutgoingContainer()));
+
+    // EOF
+
+    assertFalse(scan.next());
+    assertEquals(0, scan.batchAccessor().getRowCount());
+
+    // Next again: no-op
+
+    assertFalse(scan.next());
+    scanFixture.close();
+
+    // Close again: no-op
+
+    scan.close();
+  }
+
+  /**
+   * Test EOF on the first batch. Is allowed, but will result in the scan operator
+   * passing a null batch to the parent.
+   */
+
+  @Test
+  public void testEOFOnSchema() {
+    MockNullEarlySchemaReader reader = new MockNullEarlySchemaReader();
+
+    ScanFixture scanFixture = simpleFixture(reader);
+    ScanOperatorExec scan = scanFixture.scanOp;
+
+    // EOF
+
+    assertFalse(scan.buildSchema());
+    assertTrue(reader.closeCalled);
+    assertEquals(0, scan.batchAccessor().getRowCount());
+
+    scanFixture.close();
+  }
+
+  @Test
+  public void testEOFOnFirstBatch() {
+    MockEarlySchemaReader reader = new MockEarlySchemaReader();
+    reader.batchLimit = 0;
+
+    ScanFixture scanFixture = simpleFixture(reader);
+    ScanOperatorExec scan = scanFixture.scanOp;
+    assertTrue(scan.buildSchema());
+
+    // EOF
+
+    assertFalse(scan.next());
+    assertTrue(reader.closeCalled);
+    assertEquals(0, scan.batchAccessor().getRowCount());
+
+    scanFixture.close();
+  }
+
+  /**
+   * Test normal case with multiple readers. These return
+   * the same schema, so no schema change.
+   */
+
+  @Test
+  public void testMultipleReaders() {
+    MockNullEarlySchemaReader nullReader = new MockNullEarlySchemaReader();
+
+    MockEarlySchemaReader reader1 = new MockEarlySchemaReader();
+    reader1.batchLimit = 2;
+
+    MockEarlySchemaReader reader2 = new MockEarlySchemaReader();
+    reader2.batchLimit = 2;
+    reader2.startIndex = 100;
+
+    ScanFixture scanFixture = simpleFixture(nullReader, reader1, reader2);
+    ScanOperatorExec scan = scanFixture.scanOp;
+
+    // First batch, schema only.
+
+    assertTrue(scan.buildSchema());
+    assertEquals(1, scan.batchAccessor().schemaVersion());
+    scan.batchAccessor().release();
+
+    // Second batch.
+
+    assertTrue(scan.next());
+    assertEquals(1, reader1.batchCount);
+    assertEquals(1, scan.batchAccessor().schemaVersion());
+    verifyBatch(0, scan.batchAccessor().getOutgoingContainer());
+
+    // Third batch.
+
+    assertTrue(scan.next());
+    assertEquals(2, reader1.batchCount);
+    assertEquals(1, scan.batchAccessor().schemaVersion());
+    verifyBatch(20, scan.batchAccessor().getOutgoingContainer());
+
+    // Second reader. First batch includes data, no special first-batch
+    // handling for the second reader.
+
+    assertFalse(reader1.closeCalled);
+    assertFalse(reader2.openCalled);
+    assertTrue(scan.next());
+    assertTrue(reader1.closeCalled);
+    assertTrue(reader2.openCalled);
+    assertEquals(1, reader2.batchCount);
+    assertEquals(1, scan.batchAccessor().schemaVersion());
+    verifyBatch(100, scan.batchAccessor().getOutgoingContainer());
+
+    // Second batch from second reader.
+
+    assertTrue(scan.next());
+    assertEquals(2, reader2.batchCount);
+    assertEquals(1, scan.batchAccessor().schemaVersion());
+    verifyBatch(120, scan.batchAccessor().getOutgoingContainer());
+
+    // EOF
+
+    assertFalse(scan.next());
+    assertTrue(reader2.closeCalled);
+    assertEquals(0, scan.batchAccessor().getRowCount());
+
+    scanFixture.close();
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOperExecLateSchema.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOperExecLateSchema.java
new file mode 100644
index 0000000..ccacdae
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOperExecLateSchema.java
@@ -0,0 +1,402 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.scan;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.drill.categories.RowSetTests;
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.ScanTestUtils.ScanFixture;
+import org.apache.drill.exec.physical.impl.scan.framework.SchemaNegotiator;
+import org.apache.drill.exec.physical.rowSet.RowSetLoader;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.metadata.ColumnBuilder;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.test.rowSet.RowSetComparison;
+import org.apache.drill.test.rowSet.RowSetUtilities;
+import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test "late schema" readers: those like JSON that discover their schema
+ * as they read data.
+ */
+@Category(RowSetTests.class)
+public class TestScanOperExecLateSchema extends BaseScanOperatorExecTest {
+
+  /**
+   * "Late schema" reader, meaning that the reader does not know the schema on
+   * open, but must "discover" it when reading data.
+   */
+
+  private static class MockLateSchemaReader extends BaseMockBatchReader {
+
+    public boolean returnDataOnFirst;
+
+    @Override
+    public boolean open(SchemaNegotiator schemaNegotiator) {
+
+      // No schema or file, just build the table loader.
+
+      tableLoader = schemaNegotiator.build();
+      openCalled = true;
+      return true;
+    }
+
+    @Override
+    public boolean next() {
+      batchCount++;
+      if (batchCount > batchLimit) {
+        return false;
+      } else if (batchCount == 1) {
+
+        // On first batch, pretend to discover the schema.
+
+        RowSetLoader rowSet = tableLoader.writer();
+        MaterializedField a = SchemaBuilder.columnSchema("a", MinorType.INT, DataMode.REQUIRED);
+        rowSet.addColumn(a);
+        MaterializedField b = new ColumnBuilder("b", MinorType.VARCHAR)
+            .setMode(DataMode.OPTIONAL)
+            .setWidth(10)
+            .build();
+        rowSet.addColumn(b);
+        if (! returnDataOnFirst) {
+          return true;
+        }
+      }
+
+      makeBatch();
+      return true;
+    }
+  }
+
+  /**
+   * Most basic test of a reader that discovers its schema as it goes along.
+   * The purpose is to validate the most basic life-cycle steps before trying
+   * more complex variations.
+   */
+
+  @Test
+  public void testLateSchemaLifecycle() {
+
+    // Create a mock reader, return two batches: one schema-only, another with data.
+
+    MockLateSchemaReader reader = new MockLateSchemaReader();
+    reader.batchLimit = 2;
+    reader.returnDataOnFirst = false;
+
+    // Create the scan operator
+
+    ScanFixture scanFixture = simpleFixture(reader);
+    ScanOperatorExec scan = scanFixture.scanOp;
+
+    // Standard startup
+
+    assertFalse(reader.openCalled);
+
+    // First batch: build schema. The reader does not help: it returns an
+    // empty first batch.
+
+    assertTrue(scan.buildSchema());
+    assertTrue(reader.openCalled);
+    assertEquals(1, reader.batchCount);
+    assertEquals(0, scan.batchAccessor().getRowCount());
+
+    // Create the expected result.
+
+    SingleRowSet expected = makeExpected(20);
+    RowSetComparison verifier = new RowSetComparison(expected);
+    assertEquals(expected.batchSchema(), scan.batchAccessor().getSchema());
+
+    // Next call, return with data.
+
+    assertTrue(scan.next());
+    verifier.verifyAndClearAll(fixture.wrap(scan.batchAccessor().getOutgoingContainer()));
+
+    // EOF
+
+    assertFalse(scan.next());
+    assertTrue(reader.closeCalled);
+    assertEquals(0, scan.batchAccessor().getRowCount());
+
+    scanFixture.close();
+  }
+
+  /**
+   * Test the case that a late scan operator is closed before
+   * the first reader is opened.
+   */
+
+  @Test
+  public void testLateSchemaEarlyClose() {
+
+    // Create a mock reader, return two batches: one schema-only, another with data.
+
+    MockLateSchemaReader reader = new MockLateSchemaReader();
+    reader.batchLimit = 2;
+    reader.returnDataOnFirst = false;
+
+    // Create the scan operator
+
+    ScanFixture scanFixture = simpleFixture(reader);
+
+    // Reader never opened.
+
+    scanFixture.close();
+    assertFalse(reader.openCalled);
+    assertEquals(0, reader.batchCount);
+    assertFalse(reader.closeCalled);
+  }
+
+  /**
+   * Test the case that a late schema reader is closed after discovering
+   * schema, before any calls to next().
+   */
+
+  @Test
+  public void testLateSchemaEarlyReaderClose() {
+
+    // Create a mock reader, return two batches: one schema-only, another with data.
+
+    MockLateSchemaReader reader = new MockLateSchemaReader();
+    reader.batchLimit = 2;
+    reader.returnDataOnFirst = false;
+
+    // Create the scan operator
+
+    ScanFixture scanFixture = simpleFixture(reader);
+    ScanOperatorExec scan = scanFixture.scanOp;
+
+    // Get the schema as above.
+
+    assertTrue(scan.buildSchema());
+
+    // No lookahead batch created.
+
+    scanFixture.close();
+    assertEquals(1, reader.batchCount);
+    assertTrue(reader.closeCalled);
+  }
+
+  /**
+   * Test the case that a late schema reader is closed before
+   * consuming the look-ahead batch used to infer schema.
+   */
+
+  @Test
+  public void testLateSchemaEarlyCloseWithData() {
+
+    // Create a mock reader, return two batches: one schema-only, another with data.
+
+    MockLateSchemaReader reader = new MockLateSchemaReader();
+    reader.batchLimit = 2;
+    reader.returnDataOnFirst = true;
+
+    // Create the scan operator
+
+    ScanFixture scanFixture = simpleFixture(reader);
+    ScanOperatorExec scan = scanFixture.scanOp;
+
+    // Get the schema as above.
+
+    assertTrue(scan.buildSchema());
+
+    // Lookahead batch created.
+
+    scanFixture.close();
+    assertEquals(1, reader.batchCount);
+    assertTrue(reader.closeCalled);
+  }
+
+  /**
+   * Test a late-schema source that has no file information.
+   * (Like a Hive or JDBC data source.)
+   */
+
+  @Test
+  public void testLateSchemaLifecycleNoFile() {
+
+    // Create a mock reader, return two batches: one schema-only, another with data.
+
+    MockLateSchemaReader reader = new MockLateSchemaReader();
+    reader.batchLimit = 2;
+    reader.returnDataOnFirst = false;
+
+    // Create the scan operator
+
+    ScanFixture scanFixture = simpleFixture(reader);
+    ScanOperatorExec scan = scanFixture.scanOp;
+
+    // Standard startup
+
+    assertFalse(reader.openCalled);
+
+    // First batch: build schema. The reader helps: it returns an
+    // empty first batch.
+
+    assertTrue(scan.buildSchema());
+    assertTrue(reader.openCalled);
+    assertEquals(1, reader.batchCount);
+    assertEquals(0, scan.batchAccessor().getRowCount());
+
+    // Create the expected result.
+
+    SingleRowSet expected = makeExpected(20);
+    RowSetComparison verifier = new RowSetComparison(expected);
+    assertEquals(expected.batchSchema(), scan.batchAccessor().getSchema());
+
+    // Next call, return with data.
+
+    assertTrue(scan.next());
+    verifier.verifyAndClearAll(fixture.wrap(scan.batchAccessor().getOutgoingContainer()));
+
+    // EOF
+
+    assertFalse(scan.next());
+    assertTrue(reader.closeCalled);
+    assertEquals(0, scan.batchAccessor().getRowCount());
+
+    scanFixture.close();
+  }
+
+  @Test
+  public void testLateSchemaNoData() {
+
+    // Create a mock reader, return two batches: one schema-only, another with data.
+
+    MockLateSchemaReader reader = new MockLateSchemaReader();
+    reader.batchLimit = 0;
+    reader.returnDataOnFirst = false;
+
+    // Create the scan operator
+
+    ScanFixture scanFixture = simpleFixture(reader);
+    ScanOperatorExec scan = scanFixture.scanOp;
+
+    // Standard startup
+
+    assertFalse(reader.openCalled);
+
+    // First batch: EOF.
+
+    assertFalse(scan.buildSchema());
+    assertTrue(reader.openCalled);
+    assertTrue(reader.closeCalled);
+    scanFixture.close();
+  }
+
+  @Test
+  public void testLateSchemaDataOnFirst() {
+
+    // Create a mock reader, return two batches: one schema-only, another with data.
+
+    MockLateSchemaReader reader = new MockLateSchemaReader();
+    reader.batchLimit = 1;
+    reader.returnDataOnFirst = true;
+
+    // Create the scan operator
+
+    ScanFixture scanFixture = simpleFixture(reader);
+    ScanOperatorExec scan = scanFixture.scanOp;
+
+    // Standard startup
+
+    assertFalse(reader.openCalled);
+
+    // First batch: build schema. The reader helps: it returns an
+    // empty first batch.
+
+    assertTrue(scan.buildSchema());
+    assertTrue(reader.openCalled);
+    assertEquals(1, reader.batchCount);
+    assertEquals(0, scan.batchAccessor().getRowCount());
+
+    SingleRowSet expected = makeExpected();
+    RowSetComparison verifier = new RowSetComparison(expected);
+    assertEquals(expected.batchSchema(), scan.batchAccessor().getSchema());
+
+    // Next call, return with data.
+
+    assertTrue(scan.next());
+    verifier.verifyAndClearAll(fixture.wrap(scan.batchAccessor().getOutgoingContainer()));
+
+    // EOF
+
+    assertFalse(scan.next());
+    assertTrue(reader.closeCalled);
+    assertEquals(0, scan.batchAccessor().getRowCount());
+
+    scanFixture.close();
+  }
+
+  /**
+  * Test the case where the reader does not play the "first batch contains
+  * only schema" game, and instead returns data. The Scan operator will
+  * split the first batch into two: one with schema only, another with
+  * data.
+  */
+
+ @Test
+ public void testNonEmptyFirstBatch() {
+   SingleRowSet expected = makeExpected();
+
+   MockLateSchemaReader reader = new MockLateSchemaReader();
+   reader.batchLimit = 2;
+   reader.returnDataOnFirst = true;
+
+   ScanFixture scanFixture = simpleFixture(reader);
+   ScanOperatorExec scan = scanFixture.scanOp;
+
+   // First batch. The reader returns a non-empty batch. The scan
+   // operator strips off the schema and returns just that.
+
+   assertTrue(scan.buildSchema());
+   assertEquals(1, reader.batchCount);
+   assertEquals(expected.batchSchema(), scan.batchAccessor().getSchema());
+   assertEquals(0, scan.batchAccessor().getRowCount());
+   scan.batchAccessor().release();
+
+   // Second batch. Returns the "look-ahead" batch returned by
+   // the reader earlier.
+
+   assertTrue(scan.next());
+   assertEquals(1, reader.batchCount);
+   RowSetUtilities.verify(expected,
+     fixture.wrap(scan.batchAccessor().getOutgoingContainer()));
+
+   // Third batch, normal case.
+
+   assertTrue(scan.next());
+   assertEquals(2, reader.batchCount);
+   RowSetUtilities.verify(makeExpected(20),
+     fixture.wrap(scan.batchAccessor().getOutgoingContainer()));
+
+   // EOF
+
+   assertFalse(scan.next());
+   assertTrue(reader.closeCalled);
+   assertEquals(0, scan.batchAccessor().getRowCount());
+
+   scanFixture.close();
+ }
+
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOperExecOuputSchema.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOperExecOuputSchema.java
new file mode 100644
index 0000000..4cd7fe6
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOperExecOuputSchema.java
@@ -0,0 +1,253 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.scan;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.drill.categories.RowSetTests;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.ScanTestUtils.ScanFixture;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.impl.scan.framework.SchemaNegotiator;
+import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
+import org.apache.drill.exec.physical.rowSet.RowSetLoader;
+import org.apache.drill.exec.physical.rowSet.impl.RowSetTestUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.test.rowSet.RowSetUtilities;
+import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test the addition of an output schema to a reader. The output schema
+ * defines the schema to be output from the scan operator, and forces
+ * conversions between reader and output data types.
+ */
+@Category(RowSetTests.class)
+public class TestScanOperExecOuputSchema extends BaseScanOperatorExecTest {
+
+  private static class MockSimpleReader implements ManagedReader<SchemaNegotiator> {
+
+    private ResultSetLoader tableLoader;
+
+    @Override
+    public boolean open(SchemaNegotiator schemaNegotiator) {
+      TupleMetadata schema = new SchemaBuilder()
+          // Schema provided in test
+          .add("a", MinorType.VARCHAR)
+          // No schema provided
+          .add("b", MinorType.VARCHAR)
+          // No schema and not projected in test
+          .add("c", MinorType.VARCHAR)
+          .buildSchema();
+      schemaNegotiator.setTableSchema(schema, true);
+      tableLoader = schemaNegotiator.build();
+      return true;
+    }
+
+    @Override
+    public boolean next() {
+
+      // First batch is schema only, we provide batch 2
+
+      if (tableLoader.batchCount() > 1) {
+        return false;
+      }
+      RowSetLoader writer = tableLoader.writer();
+      writer.start();
+      writer.scalar(0).setString("10");
+      writer.scalar(1).setString("foo");
+      writer.scalar(2).setString("bar");
+      writer.save();
+      return true;
+    }
+
+    @Override
+    public void close() { }
+  }
+
+  /**
+   * Test an output schema.
+   * <ul>
+   * <li>Column a has an input type of VARCHAR, and output type of INT,
+   * and the framework will insert an implicit conversion.</li>
+   * <li>Column b has an output type of BIGINT, is projected, but is
+   * not provided by the reader. It will use the default value of 20L.</li>
+   * <li>Column c is not in the output schema, is not provided by the
+   * reader, but is projected, so it will use the default null type
+   * of VARCHAR, with a null value.</li>
+   * </ul>
+   */
+
+  @Test
+  public void testOutputSchema() {
+    TupleMetadata outputSchema = new SchemaBuilder()
+        .add("a", MinorType.INT) // Projected, in reader
+        .add("d", MinorType.BIGINT) // Projected, not in reader
+        .add("e", MinorType.BIGINT) // Not projected, not in reader
+        .buildSchema();
+    outputSchema.metadata("d").setDefaultValue("20");
+    outputSchema.metadata("e").setDefaultValue("30");
+
+    BaseScanFixtureBuilder builder = new BaseScanFixtureBuilder();
+    // Project (schema + reader), (reader only), (schema only), (neither)
+    builder.setProjection(new String[]{"a", "b", "d", "f"});
+    builder.addReader(new MockSimpleReader());
+    builder.builder.setOutputSchema(outputSchema);
+    builder.builder.setNullType(Types.optional(MinorType.VARCHAR));
+    ScanFixture scanFixture = builder.build();
+    ScanOperatorExec scan = scanFixture.scanOp;
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .add("b", MinorType.VARCHAR)
+        .add("d", MinorType.BIGINT)
+        .addNullable("f", MinorType.VARCHAR)
+        .buildSchema();
+
+    // Initial schema
+
+    assertTrue(scan.buildSchema());
+    {
+      SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
+           .build();
+      RowSetUtilities.verify(expected,
+          fixture.wrap(scan.batchAccessor().getOutgoingContainer()));
+    }
+
+    // Batch with defaults and null types
+
+    assertTrue(scan.next());
+    {
+      SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
+          .addRow(10, "foo", 20L, null)
+          .build();
+      RowSetUtilities.verify(expected,
+          fixture.wrap(scan.batchAccessor().getOutgoingContainer()));
+    }
+
+    assertFalse(scan.next());
+    scanFixture.close();
+  }
+
+  @Test
+  public void testOutputSchemaWithWildcard() {
+    TupleMetadata outputSchema = new SchemaBuilder()
+        .add("a", MinorType.INT) // Projected, in reader
+        .add("d", MinorType.BIGINT) // Projected, not in reader
+        .add("e", MinorType.BIGINT) // Not projected, not in reader
+        .buildSchema();
+    outputSchema.metadata("d").setDefaultValue("20");
+    outputSchema.metadata("e").setDefaultValue("30");
+
+    BaseScanFixtureBuilder builder = new BaseScanFixtureBuilder();
+    // Project (schema + reader), (reader only), (schema only), (neither)
+    builder.setProjection(RowSetTestUtils.projectAll());
+    builder.addReader(new MockSimpleReader());
+    builder.builder.setOutputSchema(outputSchema);
+    builder.builder.setNullType(Types.optional(MinorType.VARCHAR));
+    ScanFixture scanFixture = builder.build();
+    ScanOperatorExec scan = scanFixture.scanOp;
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .add("d", MinorType.BIGINT)
+        .add("e", MinorType.BIGINT)
+        .add("b", MinorType.VARCHAR)
+        .add("c", MinorType.VARCHAR)
+        .buildSchema();
+
+    // Initial schema
+
+    assertTrue(scan.buildSchema());
+    {
+      SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
+           .build();
+      RowSetUtilities.verify(expected,
+          fixture.wrap(scan.batchAccessor().getOutgoingContainer()));
+    }
+
+    // Batch with defaults and null types
+
+    assertTrue(scan.next());
+    {
+      SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
+          .addRow(10, 20L, 30L, "foo", "bar")
+          .build();
+      RowSetUtilities.verify(expected,
+          fixture.wrap(scan.batchAccessor().getOutgoingContainer()));
+    }
+
+    assertFalse(scan.next());
+    scanFixture.close();
+  }
+
+  @Test
+  public void testStrictOutputSchemaWithWildcard() {
+    TupleMetadata outputSchema = new SchemaBuilder()
+        .add("a", MinorType.INT) // Projected, in reader
+        .add("d", MinorType.BIGINT) // Projected, not in reader
+        .add("e", MinorType.BIGINT) // Not projected, not in reader
+        .buildSchema();
+    outputSchema.metadata("d").setDefaultValue("20");
+    outputSchema.metadata("e").setDefaultValue("30");
+    outputSchema.setProperty(TupleMetadata.IS_STRICT_SCHEMA_PROP, Boolean.TRUE.toString());
+
+    BaseScanFixtureBuilder builder = new BaseScanFixtureBuilder();
+    // Project (schema + reader), (reader only), (schema only), (neither)
+    builder.setProjection(RowSetTestUtils.projectAll());
+    builder.addReader(new MockSimpleReader());
+    builder.builder.setOutputSchema(outputSchema);
+    builder.builder.setNullType(Types.optional(MinorType.VARCHAR));
+    ScanFixture scanFixture = builder.build();
+    ScanOperatorExec scan = scanFixture.scanOp;
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .add("d", MinorType.BIGINT)
+        .add("e", MinorType.BIGINT)
+         .buildSchema();
+
+    // Initial schema
+
+    assertTrue(scan.buildSchema());
+    {
+      SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
+           .build();
+      RowSetUtilities.verify(expected,
+          fixture.wrap(scan.batchAccessor().getOutgoingContainer()));
+    }
+
+    // Batch with defaults and null types
+
+    assertTrue(scan.next());
+    {
+      SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
+          .addRow(10, 20L, 30L)
+          .build();
+      RowSetUtilities.verify(expected,
+          fixture.wrap(scan.batchAccessor().getOutgoingContainer()));
+    }
+
+    assertFalse(scan.next());
+    scanFixture.close();
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOperExecOverflow.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOperExecOverflow.java
new file mode 100644
index 0000000..2f67d58
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOperExecOverflow.java
@@ -0,0 +1,195 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.scan;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Arrays;
+
+import org.apache.drill.categories.RowSetTests;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.ScanTestUtils.ScanFixture;
+import org.apache.drill.exec.physical.impl.scan.framework.SchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.project.ScanSchemaOrchestrator;
+import org.apache.drill.exec.physical.rowSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test vector overflow in the context of the scan operator.
+ */
+@Category(RowSetTests.class)
+public class TestScanOperExecOverflow extends BaseScanOperatorExecTest {
+
+  /**
+   * Mock reader that produces "jumbo" batches that cause a vector to
+   * fill and a row to overflow from one batch to the next.
+   */
+
+  private static class OverflowReader extends BaseMockBatchReader {
+
+    private final String value;
+    public int rowCount;
+    /**
+     * If true, the reader will report EOF after filling a batch
+     * to overflow. This simulates the corner case in which a reader
+     * has, say, 1000 rows, hits overflow on row 1000, then declares
+     * it has nothing more to read.
+     * <p>
+     * If false, reports EOF on a call to next() without reading more
+     * rows. The overlow row from the prior batch still exists in
+     * the result set loader.
+     */
+    public boolean reportEofWithOverflow;
+
+    public OverflowReader() {
+      char buf[] = new char[512];
+      Arrays.fill(buf, 'x');
+      value = new String(buf);
+    }
+
+    @Override
+    public boolean open(SchemaNegotiator schemaNegotiator) {
+      openCalled = true;
+      TupleMetadata schema = new SchemaBuilder()
+          .add("a", MinorType.VARCHAR)
+          .buildSchema();
+      schemaNegotiator.setTableSchema(schema, true);
+      tableLoader = schemaNegotiator.build();
+      return true;
+    }
+
+    @Override
+    public boolean next() {
+      batchCount++;
+      if (batchCount > batchLimit) {
+        return false;
+      }
+
+      RowSetLoader writer = tableLoader.writer();
+      while (! writer.isFull()) {
+        writer.start();
+        writer.scalar(0).setString(value);
+        writer.save();
+        rowCount++;
+      }
+
+      // The vector overflowed on the last row. But, we still had to write the row.
+      // The row is tucked away in the loader to appear as the first row in
+      // the next batch.
+      //
+      // Depending on the flag set by the test routine, either report the EOF
+      // during this read, or report it next time around.
+
+      return reportEofWithOverflow
+          ? batchCount < batchLimit
+          : true;
+    }
+  }
+
+  /**
+   * Test multiple readers, with one of them creating "jumbo" batches
+   * that overflow. Specifically, test a corner case. A batch ends right
+   * at file EOF, but that last batch overflowed.
+   */
+
+  @Test
+  public void testMultipleReadersWithOverflow() {
+    runOverflowTest(false);
+    runOverflowTest(true);
+  }
+
+  private void runOverflowTest(boolean eofWithData) {
+    OverflowReader reader1 = new OverflowReader();
+    reader1.batchLimit = 2;
+    reader1.reportEofWithOverflow = eofWithData;
+    MockEarlySchemaReader reader2 = new MockEarlySchemaReader();
+    reader2.batchLimit = 2;
+
+    BaseScanFixtureBuilder builder = new BaseScanFixtureBuilder();
+    builder.projectAll();
+    builder.addReader(reader1);
+    builder.addReader(reader2);
+
+    // Want overflow, set size and row counts at their limits.
+
+    builder.builder.setBatchByteLimit(ScanSchemaOrchestrator.MAX_BATCH_BYTE_SIZE);
+    builder.builder.setBatchRecordLimit(ScanSchemaOrchestrator.MAX_BATCH_ROW_COUNT);
+    ScanFixture scanFixture = builder.build();
+    ScanOperatorExec scan = scanFixture.scanOp;
+
+    assertTrue(scan.buildSchema());
+    assertEquals(1, scan.batchAccessor().schemaVersion());
+    scan.batchAccessor().release();
+
+    // Second batch. Should be 1 less than the reader's row
+    // count because the loader has its own one-row lookahead batch.
+
+    assertTrue(scan.next());
+    assertEquals(1, reader1.batchCount);
+    assertEquals(1, scan.batchAccessor().schemaVersion());
+    int prevRowCount = scan.batchAccessor().getRowCount();
+    assertEquals(reader1.rowCount - 1, prevRowCount);
+    scan.batchAccessor().release();
+
+    // Third batch, adds more data to the lookahead batch. Also overflows
+    // so returned records is one less than total produced so far minus
+    // those returned earlier.
+
+    assertTrue(scan.next());
+    assertEquals(2, reader1.batchCount);
+    assertEquals(1, scan.batchAccessor().schemaVersion());
+    assertEquals(reader1.rowCount - prevRowCount - 1, scan.batchAccessor().getRowCount());
+    scan.batchAccessor().release();
+    int prevReaderRowCount = reader1.rowCount;
+
+    // Third batch. Returns the overflow row from the second batch of
+    // the first reader.
+
+    assertTrue(scan.next());
+    assertEquals(eofWithData ? 2 : 3, reader1.batchCount);
+    assertEquals(1, scan.batchAccessor().schemaVersion());
+    assertEquals(1, scan.batchAccessor().getRowCount());
+    assertEquals(prevReaderRowCount, reader1.rowCount);
+    scan.batchAccessor().release();
+
+    // Second reader.
+
+    assertTrue(scan.next());
+    assertEquals(2, scan.batchAccessor().schemaVersion());
+    scan.batchAccessor().release();
+
+    // Second batch from second reader.
+
+    assertTrue(scan.next());
+    assertEquals(2, reader2.batchCount);
+    assertEquals(2, scan.batchAccessor().schemaVersion());
+    scan.batchAccessor().release();
+
+    // EOF
+
+    assertFalse(scan.next());
+    assertEquals(0, scan.batchAccessor().getRowCount());
+    scanFixture.close();
+  }
+
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOperExecSmoothing.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOperExecSmoothing.java
new file mode 100644
index 0000000..575e541
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOperExecSmoothing.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.physical.impl.scan;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.drill.categories.RowSetTests;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.ScanTestUtils.ScanFixture;
+import org.apache.drill.exec.physical.impl.scan.framework.SchemaNegotiator;
+import org.apache.drill.exec.physical.rowSet.RowSetLoader;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.test.rowSet.RowSetUtilities;
+import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test the ability of the scan operator to "absorb" schema changes by
+ * "smoothing" out data types and modes across readers. Schema smoothing
+ * is helpful when there is no output schema, but only works within the
+ * context of a single scan operator: it cannot help when a query has
+ * multiple scans, each in its own fragment.
+ */
+@Category(RowSetTests.class)
+public class TestScanOperExecSmoothing extends BaseScanOperatorExecTest {
+
+  private static class MockEarlySchemaReader2 extends MockEarlySchemaReader {
+
+    @Override
+    public boolean open(SchemaNegotiator schemaNegotiator) {
+      openCalled = true;
+      TupleMetadata schema = new SchemaBuilder()
+          .add("a", MinorType.VARCHAR)
+          .addNullable("b", MinorType.VARCHAR, 10)
+          .buildSchema();
+      schemaNegotiator.setTableSchema(schema, true);
+      schemaNegotiator.build();
+      tableLoader = schemaNegotiator.build();
+      return true;
+    }
+
+    @Override
+    protected void writeRow(RowSetLoader writer, int col1, String col2) {
+      writer.start();
+      if (writer.column(0) != null) {
+        writer.scalar(0).setString(Integer.toString(col1));
+      }
+      if (writer.column(1) != null) {
+        writer.scalar(1).setString(col2);
+      }
+      writer.save();
+    }
+  }
+
+  private static class MockOneColEarlySchemaReader extends BaseMockBatchReader {
+
+    @Override
+    public boolean open(SchemaNegotiator schemaNegotiator) {
+      openCalled = true;
+      TupleMetadata schema = new SchemaBuilder()
+          .add("a", MinorType.INT)
+          .buildSchema();
+      schemaNegotiator.setTableSchema(schema, true);
+      tableLoader = schemaNegotiator.build();
+      return true;
+    }
+
+    @Override
+    public boolean next() {
+      batchCount++;
+      if (batchCount > batchLimit) {
+        return false;
+      }
+
+      makeBatch();
+      return true;
+    }
+
+    @Override
+    protected void writeRow(RowSetLoader writer, int col1, String col2) {
+      writer.start();
+      if (writer.column(0) != null) {
+        writer.scalar(0).setInt(col1 + 1);
+      }
+      writer.save();
+    }
+  }
+
+  /**
+   * Multiple readers with a schema change between them.
+   */
+
+  @Test
+  public void testSchemaChange() {
+    MockEarlySchemaReader reader1 = new MockEarlySchemaReader();
+    reader1.batchLimit = 2;
+    MockEarlySchemaReader reader2 = new MockEarlySchemaReader2();
+    reader2.batchLimit = 2;
+
+    ScanFixture scanFixture = simpleFixture(reader1, reader2);
+    ScanOperatorExec scan = scanFixture.scanOp;
+
+    // Build schema
+
+    assertTrue(scan.buildSchema());
+    assertEquals(1, scan.batchAccessor().schemaVersion());
+    scan.batchAccessor().release();
+
+    // First batch
+
+    assertTrue(scan.next());
+    assertEquals(1, scan.batchAccessor().schemaVersion());
+    scan.batchAccessor().release();
+
+    // Second batch
+
+    assertTrue(scan.next());
+    assertEquals(1, scan.batchAccessor().schemaVersion());
+    scan.batchAccessor().release();
+
+    // Second reader.
+
+    BatchSchema expectedSchema2 = new SchemaBuilder()
+        .add("a", MinorType.VARCHAR)
+        .addNullable("b", MinorType.VARCHAR, 10)
+        .build();
+
+    assertTrue(scan.next());
+    assertEquals(2, scan.batchAccessor().schemaVersion());
+    SingleRowSet expected = fixture.rowSetBuilder(expectedSchema2)
+        .addRow("10", "fred")
+        .addRow("20", "wilma")
+        .build();
+    RowSetUtilities.verify(expected,
+      fixture.wrap(scan.batchAccessor().getOutgoingContainer()));
+
+    // Second batch from second reader.
+
+    assertTrue(scan.next());
+    assertEquals(2, scan.batchAccessor().schemaVersion());
+    expected = fixture.rowSetBuilder(expectedSchema2)
+        .addRow("30", "fred")
+        .addRow("40", "wilma")
+        .build();
+    RowSetUtilities.verify(expected,
+      fixture.wrap(scan.batchAccessor().getOutgoingContainer()));
+
+    // EOF
+
+    assertFalse(scan.next());
+    assertTrue(reader2.closeCalled);
+    assertEquals(0, scan.batchAccessor().getRowCount());
+
+    scanFixture.close();
+  }
+
+ /**
+   * Test the ability of the scan operator to "smooth" out schema changes
+   * by reusing the type from a previous reader, if known. That is,
+   * given three readers:<br>
+   * (a, b)<br>
+   * (b)<br>
+   * (a, b)<br>
+   * Then the type of column a should be preserved for the second reader that
+   * does not include a. This works if a is nullable. If so, a's type will
+   * be used for the empty column, rather than the usual nullable int.
+   * <p>
+   * Full testing of smoothing is done in
+   * {#link TestScanProjector}. Here we just make sure that the
+   * smoothing logic is available via the scan operator.
+   */
+
+  @Test
+  public void testSchemaSmoothing() {
+
+    // Reader returns (a, b)
+    MockEarlySchemaReader reader1 = new MockEarlySchemaReader();
+    reader1.batchLimit = 1;
+
+    // Reader returns (a)
+    MockOneColEarlySchemaReader reader2 = new MockOneColEarlySchemaReader();
+    reader2.batchLimit = 1;
+    reader2.startIndex = 100;
+
+    // Reader returns (a, b)
+    MockEarlySchemaReader reader3 = new MockEarlySchemaReader();
+    reader3.batchLimit = 1;
+    reader3.startIndex = 200;
+
+    BaseScanFixtureBuilder builder = new BaseScanFixtureBuilder();
+    builder.setProjection(new String[]{"a", "b"});
+    builder.addReader(reader1);
+    builder.addReader(reader2);
+    builder.addReader(reader3);
+    ScanFixture scanFixture = builder.build();
+    ScanOperatorExec scan = scanFixture.scanOp;
+
+    // Schema based on (a, b)
+
+    assertTrue(scan.buildSchema());
+    assertEquals(1, scan.batchAccessor().schemaVersion());
+    scan.batchAccessor().release();
+
+    // Batch from (a, b) reader 1
+
+    assertTrue(scan.next());
+    assertEquals(1, scan.batchAccessor().schemaVersion());
+    verifyBatch(0, scan.batchAccessor().getOutgoingContainer());
+
+    // Batch from (a) reader 2
+    // Due to schema smoothing, b vector type is left unchanged,
+    // but is null filled.
+
+    assertTrue(scan.next());
+    assertEquals(1, scan.batchAccessor().schemaVersion());
+
+    SingleRowSet expected = fixture.rowSetBuilder(scan.batchAccessor().getSchema())
+        .addRow(111, null)
+        .addRow(121, null)
+        .build();
+    RowSetUtilities.verify(expected,
+        fixture.wrap(scan.batchAccessor().getOutgoingContainer()));
+
+    // Batch from (a, b) reader 3
+    // Recycles b again, back to being a table column.
+
+    assertTrue(scan.next());
+    assertEquals(1, scan.batchAccessor().schemaVersion());
+    verifyBatch(200, scan.batchAccessor().getOutgoingContainer());
+
+    assertFalse(scan.next());
+    scanFixture.close();
+  }
+
+
+  // TODO: Schema change in late reader
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOperatorExec.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOperatorExec.java
deleted file mode 100644
index 386849b..0000000
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOperatorExec.java
+++ /dev/null
@@ -1,1603 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.physical.impl.scan;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-import org.apache.drill.categories.RowSetTests;
-import org.apache.drill.common.exceptions.ExecutionSetupException;
-import org.apache.drill.common.exceptions.UserException;
-import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.common.types.TypeProtos.DataMode;
-import org.apache.drill.common.types.TypeProtos.MajorType;
-import org.apache.drill.common.types.TypeProtos.MinorType;
-import org.apache.drill.exec.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.framework.AbstractScanFramework;
-import org.apache.drill.exec.physical.impl.scan.framework.BasicScanFramework;
-import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
-import org.apache.drill.exec.physical.impl.scan.framework.SchemaNegotiator;
-import org.apache.drill.exec.physical.impl.scan.project.ScanSchemaOrchestrator;
-import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
-import org.apache.drill.exec.physical.rowSet.RowSetLoader;
-import org.apache.drill.exec.physical.rowSet.impl.RowSetTestUtils;
-import org.apache.drill.exec.record.BatchSchema;
-import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.record.VectorContainer;
-import org.apache.drill.exec.record.metadata.ColumnBuilder;
-import org.apache.drill.exec.record.metadata.SchemaBuilder;
-import org.apache.drill.exec.record.metadata.TupleMetadata;
-import org.apache.drill.test.SubOperatorTest;
-import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
-import org.apache.drill.test.rowSet.RowSetComparison;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-/**
- * Test of the scan operator framework. Here the focus is on the
- * implementation of the scan operator itself. This operator is
- * based on a number of lower-level abstractions, each of which has
- * its own unit tests. To make this more concrete: review the scan
- * operator code paths. Each path should be exercised by one or more
- * of the tests here. If, however, the code path depends on the
- * details of another, supporting class, then tests for that class
- * appear elsewhere.
- */
-
-@Category(RowSetTests.class)
-public class TestScanOperatorExec extends SubOperatorTest {
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestScanOperatorExec.class);
-
-  /**
-   * Base class for the "mock" readers used in this test. The mock readers
-   * follow the normal (enhanced) reader API, but instead of actually reading
-   * from a data source, they just generate data with a known schema.
-   * They also expose internal state such as identifying which methods
-   * were actually called.
-   */
-
-  private static abstract class BaseMockBatchReader implements ManagedReader<SchemaNegotiator> {
-    public boolean openCalled;
-    public boolean closeCalled;
-    public int startIndex;
-    public int batchCount;
-    public int batchLimit;
-    protected ResultSetLoader tableLoader;
-
-    protected void makeBatch() {
-      RowSetLoader writer = tableLoader.writer();
-      int offset = (batchCount - 1) * 20 + startIndex;
-      writeRow(writer, offset + 10, "fred");
-      writeRow(writer, offset + 20, "wilma");
-    }
-
-    protected void writeRow(RowSetLoader writer, int col1, String col2) {
-      writer.start();
-      if (writer.column(0) != null) {
-        writer.scalar(0).setInt(col1);
-      }
-      if (writer.column(1) != null) {
-        writer.scalar(1).setString(col2);
-      }
-      writer.save();
-    }
-
-    @Override
-    public void close() {
-      closeCalled = true;
-    }
-  }
-
-  /**
-   * "Late schema" reader, meaning that the reader does not know the schema on
-   * open, but must "discover" it when reading data.
-   */
-
-  private static class MockLateSchemaReader extends BaseMockBatchReader {
-
-    public boolean returnDataOnFirst;
-
-    @Override
-    public boolean open(SchemaNegotiator schemaNegotiator) {
-
-      // No schema or file, just build the table loader.
-
-      tableLoader = schemaNegotiator.build();
-      openCalled = true;
-      return true;
-    }
-
-    @Override
-    public boolean next() {
-      batchCount++;
-      if (batchCount > batchLimit) {
-        return false;
-      } else if (batchCount == 1) {
-
-        // On first batch, pretend to discover the schema.
-
-        RowSetLoader rowSet = tableLoader.writer();
-        MaterializedField a = SchemaBuilder.columnSchema("a", MinorType.INT, DataMode.REQUIRED);
-        rowSet.addColumn(a);
-        MaterializedField b = new ColumnBuilder("b", MinorType.VARCHAR)
-            .setMode(DataMode.OPTIONAL)
-            .setWidth(10)
-            .build();
-        rowSet.addColumn(b);
-        if (! returnDataOnFirst) {
-          return true;
-        }
-      }
-
-      makeBatch();
-      return true;
-    }
-  }
-
-  /**
-   * Mock reader that returns no schema and no records.
-   */
-
-  private static class MockNullEarlySchemaReader extends BaseMockBatchReader {
-
-    @Override
-    public boolean open(SchemaNegotiator schemaNegotiator) {
-      openCalled = true;
-      return false;
-    }
-
-    @Override
-    public boolean next() {
-      return false;
-    }
-  }
-
-  /**
-   * Mock reader that pretends to have a schema at open time
-   * like an HBase or JDBC reader.
-   */
-
-  private static class MockEarlySchemaReader extends BaseMockBatchReader {
-
-    @Override
-    public boolean open(SchemaNegotiator schemaNegotiator) {
-      openCalled = true;
-      TupleMetadata schema = new SchemaBuilder()
-          .add("a", MinorType.INT)
-          .addNullable("b", MinorType.VARCHAR, 10)
-          .buildSchema();
-      schemaNegotiator.setTableSchema(schema, true);
-      tableLoader = schemaNegotiator.build();
-      return true;
-    }
-
-    @Override
-    public boolean next() {
-      batchCount++;
-      if (batchCount > batchLimit) {
-        return false;
-      }
-
-      makeBatch();
-      return true;
-    }
-  }
-
-  private static class MockEarlySchemaReader2 extends MockEarlySchemaReader {
-
-    @Override
-    public boolean open(SchemaNegotiator schemaNegotiator) {
-      openCalled = true;
-      TupleMetadata schema = new SchemaBuilder()
-          .add("a", MinorType.VARCHAR)
-          .addNullable("b", MinorType.VARCHAR, 10)
-          .buildSchema();
-      schemaNegotiator.setTableSchema(schema, true);
-      schemaNegotiator.build();
-      tableLoader = schemaNegotiator.build();
-      return true;
-    }
-
-    @Override
-    protected void writeRow(RowSetLoader writer, int col1, String col2) {
-      writer.start();
-      if (writer.column(0) != null) {
-        writer.scalar(0).setString(Integer.toString(col1));
-      }
-      if (writer.column(1) != null) {
-        writer.scalar(1).setString(col2);
-      }
-      writer.save();
-    }
-  }
-
-  private SingleRowSet makeExpected() {
-    return makeExpected(0);
-  }
-
-  private SingleRowSet makeExpected(int offset) {
-    BatchSchema expectedSchema = new SchemaBuilder()
-        .add("a", MinorType.INT)
-        .addNullable("b", MinorType.VARCHAR, 10)
-        .build();
-    SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
-        .addRow(offset + 10, "fred")
-        .addRow(offset + 20, "wilma")
-        .build();
-    return expected;
-  }
-
-  private void verifyBatch(int offset, VectorContainer output) {
-    SingleRowSet expected = makeExpected(offset);
-    new RowSetComparison(expected)
-        .verifyAndClearAll(fixture.wrap(output));
-  }
-
-  public abstract static class AbstractScanOpFixture {
-    private OperatorContext opContext;
-    protected List<SchemaPath> projection;
-    public ScanOperatorExec scanOp;
-    private int batchByteCount;
-    private int maxRowCount;
-    private MajorType nullType;
-
-    public void projectAll() {
-      projection = RowSetTestUtils.projectAll();
-    }
-
-    public void projectAllWithMetadata(int dirs) {
-      projection = ScanTestUtils.projectAllWithMetadata(dirs);
-    }
-
-    public void setProjection(String... projCols) {
-      projection = RowSetTestUtils.projectList(projCols);
-    }
-
-    public void setProjection(List<SchemaPath> cols) {
-      projection = cols;
-    }
-
-    public void setMaxBatchByteCount(int byteCount) {
-      batchByteCount = byteCount;
-    }
-
-    public void setMaxRowCount(int rowCount) {
-      maxRowCount = rowCount;
-    }
-
-    public void setNullType(MajorType type) {
-      nullType = type;
-    }
-
-    protected void configure(AbstractScanFramework<?> framework) {
-      framework.setMaxBatchByteCount(batchByteCount);
-      framework.setMaxRowCount(maxRowCount);
-      framework.setNullType(nullType);
-    }
-
-    protected ScanOperatorExec buildScanOp(ScanOperatorEvents framework) {
-      scanOp = new ScanOperatorExec(framework);
-      Scan scanConfig = new AbstractSubScan("bob") {
-
-        @Override
-        public int getOperatorType() {
-          return 0;
-        }
-      };
-      opContext = fixture.newOperatorContext(scanConfig);
-      scanOp.bind(opContext);
-      return scanOp;
-    }
-
-    public void close() {
-      try {
-        scanOp.close();
-      } finally {
-        opContext.close();
-      }
-    }
-  }
-
-  /**
-   * Fixture to handle the boiler-plate needed to set up the components that make
-   * up a scan. (In real code, this is all done via the scan batch creator.)
-   */
-
-  public static class BasicScanOpFixture extends AbstractScanOpFixture {
-
-    public final List<ManagedReader<SchemaNegotiator>> readers = new ArrayList<>();
-    public BasicScanFramework framework;
-
-    public void addReader(ManagedReader<SchemaNegotiator> reader) {
-      readers.add(reader);
-    }
-
-    public ScanOperatorExec build() {
-      framework = new BasicScanFramework(projection, readers.iterator());
-      configure(framework);
-      return buildScanOp(framework);
-    }
-  }
-
-  /**
-   * Most basic test of a reader that discovers its schema as it goes along.
-   * The purpose is to validate the most basic life-cycle steps before trying
-   * more complex variations.
-   */
-
-  @Test
-  public void testLateSchemaLifecycle() {
-
-    // Create a mock reader, return two batches: one schema-only, another with data.
-
-    MockLateSchemaReader reader = new MockLateSchemaReader();
-    reader.batchLimit = 2;
-    reader.returnDataOnFirst = false;
-
-    // Create the scan operator
-
-    BasicScanOpFixture scanFixture = new BasicScanOpFixture();
-    scanFixture.projectAll();
-    scanFixture.addReader(reader);
-    ScanOperatorExec scan = scanFixture.build();
-
-    // Standard startup
-
-    assertFalse(reader.openCalled);
-
-    // First batch: build schema. The reader does not help: it returns an
-    // empty first batch.
-
-    assertTrue(scan.buildSchema());
-    assertTrue(reader.openCalled);
-    assertEquals(1, reader.batchCount);
-    assertEquals(0, scan.batchAccessor().getRowCount());
-
-    // Create the expected result.
-
-    SingleRowSet expected = makeExpected(20);
-    RowSetComparison verifier = new RowSetComparison(expected);
-    assertEquals(expected.batchSchema(), scan.batchAccessor().getSchema());
-
-    // Next call, return with data.
-
-    assertTrue(scan.next());
-    verifier.verifyAndClearAll(fixture.wrap(scan.batchAccessor().getOutgoingContainer()));
-
-    // EOF
-
-    assertFalse(scan.next());
-    assertTrue(reader.closeCalled);
-    assertEquals(0, scan.batchAccessor().getRowCount());
-
-    scanFixture.close();
-  }
-
-  /**
-   * Test the case that a late scan operator is closed before
-   * the first reader is opened.
-   */
-
-  @Test
-  public void testLateSchemaEarlyClose() {
-
-    // Create a mock reader, return two batches: one schema-only, another with data.
-
-    MockLateSchemaReader reader = new MockLateSchemaReader();
-    reader.batchLimit = 2;
-    reader.returnDataOnFirst = false;
-
-    // Create the scan operator
-
-    BasicScanOpFixture scanFixture = new BasicScanOpFixture();
-    scanFixture.projectAll();
-    scanFixture.addReader(reader);
-    scanFixture.build();
-
-    // Reader never opened.
-
-    scanFixture.close();
-    assertFalse(reader.openCalled);
-    assertEquals(0, reader.batchCount);
-    assertFalse(reader.closeCalled);
-  }
-
-  /**
-   * Test the case that a late schema reader is closed after discovering
-   * schema, before any calls to next().
-   */
-
-  @Test
-  public void testLateSchemaEarlyReaderClose() {
-
-    // Create a mock reader, return two batches: one schema-only, another with data.
-
-    MockLateSchemaReader reader = new MockLateSchemaReader();
-    reader.batchLimit = 2;
-    reader.returnDataOnFirst = false;
-
-    // Create the scan operator
-
-    BasicScanOpFixture scanFixture = new BasicScanOpFixture();
-    scanFixture.projectAll();
-    scanFixture.addReader(reader);
-    ScanOperatorExec scan = scanFixture.build();
-
-    // Get the schema as above.
-
-    assertTrue(scan.buildSchema());
-
-    // No lookahead batch created.
-
-    scanFixture.close();
-    assertEquals(1, reader.batchCount);
-    assertTrue(reader.closeCalled);
-  }
-
-  /**
-   * Test the case that a late schema reader is closed before
-   * consuming the look-ahead batch used to infer schema.
-   */
-
-  @Test
-  public void testLateSchemaEarlyCloseWithData() {
-
-    // Create a mock reader, return two batches: one schema-only, another with data.
-
-    MockLateSchemaReader reader = new MockLateSchemaReader();
-    reader.batchLimit = 2;
-    reader.returnDataOnFirst = true;
-
-    // Create the scan operator
-
-    BasicScanOpFixture scanFixture = new BasicScanOpFixture();
-    scanFixture.projectAll();
-    scanFixture.addReader(reader);
-    ScanOperatorExec scan = scanFixture.build();
-
-    // Get the schema as above.
-
-    assertTrue(scan.buildSchema());
-
-    // Lookahead batch created.
-
-    scanFixture.close();
-    assertEquals(1, reader.batchCount);
-    assertTrue(reader.closeCalled);
-  }
-
-  /**
-   * Pathological case that a scan operator is provided no readers.
-   * It will throw a user exception because the downstream operators
-   * can't handle this case so we choose to stop the show early to
-   * avoid getting into a strange state.
-   */
-  @Test
-  public void testNoReader() {
-
-    // Create the scan operator
-
-    BasicScanOpFixture scanFixture = new BasicScanOpFixture();
-    scanFixture.projectAll();
-    ScanOperatorExec scan = scanFixture.build();
-
-    try {
-      scan.buildSchema();
-    } catch (UserException e) {
-
-      // Expected
-
-      assertTrue(e.getCause() instanceof ExecutionSetupException);
-    }
-
-    // Must close the DAG (context and scan operator) even on failures
-
-    scanFixture.close();
-  }
-
-  /**
-   * Test a late-schema source that has no file information.
-   * (Like a Hive or JDBC data source.)
-   */
-
-  @Test
-  public void testLateSchemaLifecycleNoFile() {
-
-    // Create a mock reader, return two batches: one schema-only, another with data.
-
-    MockLateSchemaReader reader = new MockLateSchemaReader();
-    reader.batchLimit = 2;
-    reader.returnDataOnFirst = false;
-
-    // Create the scan operator
-
-    BasicScanOpFixture scanFixture = new BasicScanOpFixture();
-    scanFixture.projectAll();
-    scanFixture.addReader(reader);
-    ScanOperatorExec scan = scanFixture.build();
-
-    // Standard startup
-
-    assertFalse(reader.openCalled);
-
-    // First batch: build schema. The reader helps: it returns an
-    // empty first batch.
-
-    assertTrue(scan.buildSchema());
-    assertTrue(reader.openCalled);
-    assertEquals(1, reader.batchCount);
-    assertEquals(0, scan.batchAccessor().getRowCount());
-
-    // Create the expected result.
-
-    SingleRowSet expected = makeExpected(20);
-    RowSetComparison verifier = new RowSetComparison(expected);
-    assertEquals(expected.batchSchema(), scan.batchAccessor().getSchema());
-
-    // Next call, return with data.
-
-    assertTrue(scan.next());
-    verifier.verifyAndClearAll(fixture.wrap(scan.batchAccessor().getOutgoingContainer()));
-
-    // EOF
-
-    assertFalse(scan.next());
-    assertTrue(reader.closeCalled);
-    assertEquals(0, scan.batchAccessor().getRowCount());
-
-    scanFixture.close();
-  }
-
-  @Test
-  public void testLateSchemaNoData() {
-
-    // Create a mock reader, return two batches: one schema-only, another with data.
-
-    MockLateSchemaReader reader = new MockLateSchemaReader();
-    reader.batchLimit = 0;
-    reader.returnDataOnFirst = false;
-
-    // Create the scan operator
-
-    BasicScanOpFixture scanFixture = new BasicScanOpFixture();
-    scanFixture.projectAll();
-    scanFixture.addReader(reader);
-    ScanOperatorExec scan = scanFixture.build();
-
-    // Standard startup
-
-    assertFalse(reader.openCalled);
-
-    // First batch: EOF.
-
-    assertFalse(scan.buildSchema());
-    assertTrue(reader.openCalled);
-    assertTrue(reader.closeCalled);
-    scanFixture.close();
-  }
-
-  @Test
-  public void testLateSchemaDataOnFirst() {
-
-    // Create a mock reader, return two batches: one schema-only, another with data.
-
-    MockLateSchemaReader reader = new MockLateSchemaReader();
-    reader.batchLimit = 1;
-    reader.returnDataOnFirst = true;
-
-    // Create the scan operator
-
-    BasicScanOpFixture scanFixture = new BasicScanOpFixture();
-    scanFixture.projectAll();
-    scanFixture.addReader(reader);
-    ScanOperatorExec scan = scanFixture.build();
-
-    // Standard startup
-
-    assertFalse(reader.openCalled);
-
-    // First batch: build schema. The reader helps: it returns an
-    // empty first batch.
-
-    assertTrue(scan.buildSchema());
-    assertTrue(reader.openCalled);
-    assertEquals(1, reader.batchCount);
-    assertEquals(0, scan.batchAccessor().getRowCount());
-
-    SingleRowSet expected = makeExpected();
-    RowSetComparison verifier = new RowSetComparison(expected);
-    assertEquals(expected.batchSchema(), scan.batchAccessor().getSchema());
-
-    // Next call, return with data.
-
-    assertTrue(scan.next());
-    verifier.verifyAndClearAll(fixture.wrap(scan.batchAccessor().getOutgoingContainer()));
-
-    // EOF
-
-    assertFalse(scan.next());
-    assertTrue(reader.closeCalled);
-    assertEquals(0, scan.batchAccessor().getRowCount());
-
-    scanFixture.close();
-  }
-
-  @Test
-  public void testEarlySchemaLifecycle() {
-
-    // Create a mock reader, return two batches: one schema-only, another with data.
-
-    MockEarlySchemaReader reader = new MockEarlySchemaReader();
-    reader.batchLimit = 1;
-
-    // Create the scan operator
-
-    BasicScanOpFixture scanFixture = new BasicScanOpFixture();
-    scanFixture.projectAll();
-    scanFixture.addReader(reader);
-    ScanOperatorExec scan = scanFixture.build();
-
-    SingleRowSet expected = makeExpected();
-    RowSetComparison verifier = new RowSetComparison(expected);
-
-    // First batch: return schema.
-
-    assertTrue(scan.buildSchema());
-    assertEquals(0, reader.batchCount);
-    assertEquals(expected.batchSchema(), scan.batchAccessor().getSchema());
-    assertEquals(0, scan.batchAccessor().getRowCount());
-
-    // Next call, return with data.
-
-    assertTrue(scan.next());
-    verifier.verifyAndClearAll(fixture.wrap(scan.batchAccessor().getOutgoingContainer()));
-
-    // EOF
-
-    assertFalse(scan.next());
-    assertEquals(0, scan.batchAccessor().getRowCount());
-
-    // Next again: no-op
-
-    assertFalse(scan.next());
-    scanFixture.close();
-
-    // Close again: no-op
-
-    scan.close();
-  }
-
-  private static class MockEarlySchemaReader3 extends MockEarlySchemaReader {
-
-    @Override
-    public boolean next() {
-      if (batchCount >= batchLimit) {
-        return false;
-      }
-      batchCount++;
-
-      makeBatch();
-      return batchCount < batchLimit;
-    }
-  }
-
-  @Test
-  public void testEarlySchemaDataWithEof() {
-
-    // Create a mock reader, return two batches: one schema-only, another with data.
-
-    MockEarlySchemaReader3 reader = new MockEarlySchemaReader3();
-    reader.batchLimit = 1;
-
-    // Create the scan operator
-
-    BasicScanOpFixture scanFixture = new BasicScanOpFixture();
-    scanFixture.projectAll();
-    scanFixture.addReader(reader);
-    ScanOperatorExec scan = scanFixture.build();
-
-    SingleRowSet expected = makeExpected();
-    RowSetComparison verifier = new RowSetComparison(expected);
-
-    // First batch: return schema.
-
-    assertTrue(scan.buildSchema());
-    assertEquals(0, scan.batchAccessor().getRowCount());
-
-    // Next call, return with data.
-
-    assertTrue(scan.next());
-    verifier.verifyAndClearAll(fixture.wrap(scan.batchAccessor().getOutgoingContainer()));
-
-    // EOF
-
-    assertFalse(scan.next());
-    assertEquals(0, scan.batchAccessor().getRowCount());
-
-    // Next again: no-op
-
-    assertFalse(scan.next());
-    scanFixture.close();
-
-    // Close again: no-op
-
-    scan.close();
-  }
-
-  /**
-   * Test the case where the reader does not play the "first batch contains
-   * only schema" game, and instead returns data. The Scan operator will
-   * split the first batch into two: one with schema only, another with
-   * data.
-   */
-
-  @Test
-  public void testNonEmptyFirstBatch() {
-    SingleRowSet expected = makeExpected();
-
-    MockLateSchemaReader reader = new MockLateSchemaReader();
-    reader.batchLimit = 2;
-    reader.returnDataOnFirst = true;
-
-    BasicScanOpFixture scanFixture = new BasicScanOpFixture();
-    scanFixture.projectAll();
-    scanFixture.addReader(reader);
-    ScanOperatorExec scan = scanFixture.build();
-
-    // First batch. The reader returns a non-empty batch. The scan
-    // operator strips off the schema and returns just that.
-
-    assertTrue(scan.buildSchema());
-    assertEquals(1, reader.batchCount);
-    assertEquals(expected.batchSchema(), scan.batchAccessor().getSchema());
-    assertEquals(0, scan.batchAccessor().getRowCount());
-    scan.batchAccessor().release();
-
-    // Second batch. Returns the "look-ahead" batch returned by
-    // the reader earlier.
-
-    assertTrue(scan.next());
-    assertEquals(1, reader.batchCount);
-    new RowSetComparison(expected)
-      .verifyAndClearAll(fixture.wrap(scan.batchAccessor().getOutgoingContainer()));
-
-    // Third batch, normal case.
-
-    assertTrue(scan.next());
-    assertEquals(2, reader.batchCount);
-    new RowSetComparison(makeExpected(20))
-      .verifyAndClearAll(fixture.wrap(scan.batchAccessor().getOutgoingContainer()));
-
-    // EOF
-
-    assertFalse(scan.next());
-    assertTrue(reader.closeCalled);
-    assertEquals(0, scan.batchAccessor().getRowCount());
-
-    scanFixture.close();
-  }
-
-  /**
-   * Test EOF on the first batch. Is allowed, but will result in the scan operator
-   * passing a null batch to the parent.
-   */
-
-  @Test
-  public void testEOFOnSchema() {
-    MockNullEarlySchemaReader reader = new MockNullEarlySchemaReader();
-
-    BasicScanOpFixture scanFixture = new BasicScanOpFixture();
-    scanFixture.projectAll();
-    scanFixture.addReader(reader);
-    ScanOperatorExec scan = scanFixture.build();
-
-    // EOF
-
-    assertFalse(scan.buildSchema());
-    assertTrue(reader.closeCalled);
-    assertEquals(0, scan.batchAccessor().getRowCount());
-
-    scanFixture.close();
-  }
-
-  @Test
-  public void testEOFOnFirstBatch() {
-    MockEarlySchemaReader reader = new MockEarlySchemaReader();
-    reader.batchLimit = 0;
-
-    BasicScanOpFixture scanFixture = new BasicScanOpFixture();
-    scanFixture.projectAll();
-    scanFixture.addReader(reader);
-    ScanOperatorExec scan = scanFixture.build();
-    assertTrue(scan.buildSchema());
-
-    // EOF
-
-    assertFalse(scan.next());
-    assertTrue(reader.closeCalled);
-    assertEquals(0, scan.batchAccessor().getRowCount());
-
-    scanFixture.close();
-  }
-
-  /**
-   * Test normal case with multiple readers. These return
-   * the same schema, so no schema change.
-   */
-
-  @Test
-  public void testMultipleReaders() {
-    MockNullEarlySchemaReader nullReader = new MockNullEarlySchemaReader();
-
-    MockEarlySchemaReader reader1 = new MockEarlySchemaReader();
-    reader1.batchLimit = 2;
-
-    MockEarlySchemaReader reader2 = new MockEarlySchemaReader();
-    reader2.batchLimit = 2;
-    reader2.startIndex = 100;
-
-    BasicScanOpFixture scanFixture = new BasicScanOpFixture();
-    scanFixture.projectAll();
-    scanFixture.addReader(nullReader);
-    scanFixture.addReader(reader1);
-    scanFixture.addReader(reader2);
-    ScanOperatorExec scan = scanFixture.build();
-
-    // First batch, schema only.
-
-    assertTrue(scan.buildSchema());
-    assertEquals(1, scan.batchAccessor().schemaVersion());
-    scan.batchAccessor().release();
-
-    // Second batch.
-
-    assertTrue(scan.next());
-    assertEquals(1, reader1.batchCount);
-    assertEquals(1, scan.batchAccessor().schemaVersion());
-    verifyBatch(0, scan.batchAccessor().getOutgoingContainer());
-
-    // Third batch.
-
-    assertTrue(scan.next());
-    assertEquals(2, reader1.batchCount);
-    assertEquals(1, scan.batchAccessor().schemaVersion());
-    verifyBatch(20, scan.batchAccessor().getOutgoingContainer());
-
-    // Second reader. First batch includes data, no special first-batch
-    // handling for the second reader.
-
-    assertFalse(reader1.closeCalled);
-    assertFalse(reader2.openCalled);
-    assertTrue(scan.next());
-    assertTrue(reader1.closeCalled);
-    assertTrue(reader2.openCalled);
-    assertEquals(1, reader2.batchCount);
-    assertEquals(1, scan.batchAccessor().schemaVersion());
-    verifyBatch(100, scan.batchAccessor().getOutgoingContainer());
-
-    // Second batch from second reader.
-
-    assertTrue(scan.next());
-    assertEquals(2, reader2.batchCount);
-    assertEquals(1, scan.batchAccessor().schemaVersion());
-    verifyBatch(120, scan.batchAccessor().getOutgoingContainer());
-
-    // EOF
-
-    assertFalse(scan.next());
-    assertTrue(reader2.closeCalled);
-    assertEquals(0, scan.batchAccessor().getRowCount());
-
-    scanFixture.close();
-  }
-
-  /**
-   * Multiple readers with a schema change between them.
-   */
-
-  @Test
-  public void testSchemaChange() {
-    MockEarlySchemaReader reader1 = new MockEarlySchemaReader();
-    reader1.batchLimit = 2;
-    MockEarlySchemaReader reader2 = new MockEarlySchemaReader2();
-    reader2.batchLimit = 2;
-
-    BasicScanOpFixture scanFixture = new BasicScanOpFixture();
-    scanFixture.projectAll();
-    scanFixture.addReader(reader1);
-    scanFixture.addReader(reader2);
-    ScanOperatorExec scan = scanFixture.build();
-
-    // Build schema
-
-    assertTrue(scan.buildSchema());
-    assertEquals(1, scan.batchAccessor().schemaVersion());
-    scan.batchAccessor().release();
-
-    // First batch
-
-    assertTrue(scan.next());
-    assertEquals(1, scan.batchAccessor().schemaVersion());
-    scan.batchAccessor().release();
-
-    // Second batch
-
-    assertTrue(scan.next());
-    assertEquals(1, scan.batchAccessor().schemaVersion());
... 7251 lines suppressed ...