You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by pr...@apache.org on 2020/04/15 04:47:23 UTC

[drill] branch master updated: DRILL-7696: EVF v2 scan schema resolution

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

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


The following commit(s) were added to refs/heads/master by this push:
     new cfa0b6d  DRILL-7696: EVF v2 scan schema resolution
cfa0b6d is described below

commit cfa0b6d3c82bde159394b0090759e7a96cd5efab
Author: Paul Rogers <pa...@yahoo.com>
AuthorDate: Thu Apr 9 20:46:06 2020 -0700

    DRILL-7696: EVF v2 scan schema resolution
    
    Provides the mechanism to resolve the scan schema from a
    projection list, provided schema, early reader schema and
    actual reader schema.
---
 .../drill/common/exceptions/EmptyErrorContext.java |    2 +
 .../org/apache/drill/categories/EvfTests.java}     |   14 +-
 .../drill/exec/store/hdf5/HDF5FormatConfig.java    |    1 -
 .../impl/protocol/VectorContainerAccessor.java     |    5 +-
 .../impl/scan/file/ImplicitColumnManager.java      |    4 +-
 .../impl/scan/project/ScanLevelProjection.java     |    2 +-
 .../physical/impl/scan/v3/file/FileDescrip.java    |   74 ++
 .../impl/scan/v3/schema/AbstractSchemaTracker.java |  166 +++
 .../impl/scan/v3/schema/DynamicSchemaFilter.java   |  255 +++++
 .../scan/v3/schema/ImplicitColumnResolver.java     |  410 ++++++++
 .../impl/scan/v3/schema/MutableTupleMetadata.java  |  220 ++++
 .../impl/scan/v3/schema/MutableTupleSchema.java    |  220 ++++
 .../impl/scan/v3/schema/ProjectedColumn.java       |  190 ++++
 .../scan/v3/schema/ProjectionSchemaTracker.java    |  165 +++
 .../impl/scan/v3/schema/ScanProjectionParser.java  |  158 +++
 .../scan/v3/schema/ScanSchemaConfigBuilder.java    |  100 ++
 .../impl/scan/v3/schema/ScanSchemaResolver.java    |  370 +++++++
 .../impl/scan/v3/schema/ScanSchemaTracker.java     |  511 ++++++++++
 .../impl/scan/v3/schema/SchemaBasedTracker.java    |   94 ++
 .../physical/impl/scan/v3/schema/SchemaUtils.java  |  237 +++++
 .../physical/impl/scan/v3/schema/package-info.java |  378 +++++++
 .../physical/resultSet/impl/ColumnBuilder.java     |   33 +-
 .../exec/physical/resultSet/impl/ColumnState.java  |    3 -
 .../physical/resultSet/impl/ProjectionFilter.java  |  282 ++++--
 .../resultSet/impl/ResultSetLoaderImpl.java        |   13 +-
 .../resultSet/project/ImpliedTupleRequest.java     |    6 +
 .../physical/resultSet/project/RequestedTuple.java |    2 +
 .../resultSet/project/RequestedTupleImpl.java      |    9 +
 .../apache/drill/exec/store/ColumnExplorer.java    |   38 +-
 .../impl/scan/project/TestNullColumnLoader.java    |   33 -
 .../impl/scan/project/TestScanLevelProjection.java |   74 +-
 .../physical/impl/scan/v3/file/FileScanUtils.java  |   58 ++
 .../physical/impl/scan/v3/file/MockFileNames.java  |   22 +-
 .../impl/scan/v3/file/TestFileDescrip.java         |  149 +++
 .../scan/v3/file/TestImplicitColumnResolver.java   |  555 +++++++++++
 .../impl/scan/v3/schema/BaseTestSchemaTracker.java |   71 ++
 .../scan/v3/schema/TestDynamicSchemaFilter.java    |  179 ++++
 .../impl/scan/v3/schema/TestProjectedPath.java     |  357 +++++++
 .../impl/scan/v3/schema/TestProjectionParser.java  |  472 +++++++++
 .../impl/scan/v3/schema/TestScanSchemaTracker.java |  470 +++++++++
 .../scan/v3/schema/TestScanSchemaTrackerMaps.java  | 1052 ++++++++++++++++++++
 .../scan/v3/schema/TestSchemaTrackerDefined.java   |  276 +++++
 .../schema/TestSchemaTrackerEarlyReaderSchema.java |  172 ++++
 .../v3/schema/TestSchemaTrackerInputSchema.java    |  609 +++++++++++
 .../v3/schema/TestSchemaTrackerProjection.java     |  103 ++
 .../resultSet/impl/TestProjectionFilter.java       |  322 ++++++
 .../impl/TestResultSetLoaderProjection.java        |   17 +-
 .../exec/physical/rowSet/RowSetTestUtils.java      |    4 +-
 .../physical/rowSet/TestHyperVectorReaders.java    |   18 -
 .../exec/physical/rowSet/TestScalarAccessors.java  |    9 +-
 .../org/apache/drill/test/OperatorFixture.java     |    5 +-
 .../apache/drill/test/OperatorTestBuilderTest.java |    4 +-
 .../drill/test/rowSet/file/JsonFileBuilder.java    |   13 +-
 .../record/metadata/schema/parser/SchemaLexer.g4   |    4 +
 .../record/metadata/schema/parser/SchemaParser.g4  |    1 +
 .../main/codegen/templates/BasicTypeHelper.java    |  428 ++++----
 .../record/metadata/AbstractColumnMetadata.java    |  189 ++--
 .../record/metadata/AbstractMapColumnMetadata.java |   15 +
 .../exec/record/metadata/AbstractPropertied.java   |   33 +
 .../drill/exec/record/metadata/ColumnMetadata.java |   66 +-
 .../drill/exec/record/metadata/DynamicColumn.java  |  102 ++
 .../drill/exec/record/metadata/MapBuilder.java     |    5 +
 .../drill/exec/record/metadata/MetadataUtils.java  |   53 +
 .../record/metadata/PrimitiveColumnMetadata.java   |   33 +-
 .../metadata/RepeatedListColumnMetadata.java       |   15 +
 .../drill/exec/record/metadata/SchemaBuilder.java  |   19 +-
 .../drill/exec/record/metadata/TupleMetadata.java  |    1 +
 .../drill/exec/record/metadata/TupleNameSpace.java |    8 +-
 .../drill/exec/record/metadata/TupleSchema.java    |   16 +
 .../record/metadata/VariantColumnMetadata.java     |   15 +
 .../drill/exec/record/metadata/VariantSchema.java  |   22 +
 .../metadata/schema/parser/SchemaExprParser.java   |    2 -
 .../metadata/schema/parser/SchemaVisitor.java      |    7 +
 .../exec/record/metadata/TestTupleSchema.java      |   61 +-
 .../metadata/schema/parser/TestSchemaParser.java   |    9 +
 75 files changed, 9472 insertions(+), 638 deletions(-)

diff --git a/common/src/main/java/org/apache/drill/common/exceptions/EmptyErrorContext.java b/common/src/main/java/org/apache/drill/common/exceptions/EmptyErrorContext.java
index 9589714..a1f06d2 100644
--- a/common/src/main/java/org/apache/drill/common/exceptions/EmptyErrorContext.java
+++ b/common/src/main/java/org/apache/drill/common/exceptions/EmptyErrorContext.java
@@ -21,6 +21,8 @@ import org.apache.drill.common.exceptions.UserException.Builder;
 
 public class EmptyErrorContext implements CustomErrorContext {
 
+  public static final CustomErrorContext INSTANCE = new EmptyErrorContext();
+
   @Override
   public void addContext(Builder builder) { }
 }
diff --git a/common/src/main/java/org/apache/drill/common/exceptions/EmptyErrorContext.java b/common/src/test/java/org/apache/drill/categories/EvfTests.java
similarity index 77%
copy from common/src/main/java/org/apache/drill/common/exceptions/EmptyErrorContext.java
copy to common/src/test/java/org/apache/drill/categories/EvfTests.java
index 9589714..7bc771e 100644
--- a/common/src/main/java/org/apache/drill/common/exceptions/EmptyErrorContext.java
+++ b/common/src/test/java/org/apache/drill/categories/EvfTests.java
@@ -15,12 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.common.exceptions;
+package org.apache.drill.categories;
 
-import org.apache.drill.common.exceptions.UserException.Builder;
-
-public class EmptyErrorContext implements CustomErrorContext {
-
-  @Override
-  public void addContext(Builder builder) { }
+/**
+ * A category for tests that test the "Extended Vector Framework"
+ * (EVF): the mechanism that drives the plugin-based scan operator.
+ */
+public interface EvfTests {
+  // Junit category marker
 }
diff --git a/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/HDF5FormatConfig.java b/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/HDF5FormatConfig.java
index 770a248..87d013d 100644
--- a/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/HDF5FormatConfig.java
+++ b/contrib/format-hdf5/src/main/java/org/apache/drill/exec/store/hdf5/HDF5FormatConfig.java
@@ -43,7 +43,6 @@ public class HDF5FormatConfig implements FormatPluginConfig {
     return defaultPath;
   }
 
-
   @Override
   public boolean equals(Object obj) {
     if (this == obj) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/protocol/VectorContainerAccessor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/protocol/VectorContainerAccessor.java
index 998be89..694c91b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/protocol/VectorContainerAccessor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/protocol/VectorContainerAccessor.java
@@ -35,7 +35,6 @@ import org.apache.drill.exec.record.selection.SelectionVector4;
  * simpler than the entire {@link RecordBatch}. This implementation hosts
  * a container only.
  */
-
 public class VectorContainerAccessor implements BatchAccessor {
 
   protected VectorContainer container;
@@ -47,7 +46,6 @@ public class VectorContainerAccessor implements BatchAccessor {
    * Call this to declare a schema when there are no results to
    * report.
    */
-
   public void setSchema(VectorContainer container) {
     this.container = container;
     if (container != null) {
@@ -66,7 +64,6 @@ public class VectorContainerAccessor implements BatchAccessor {
    * @param container the container that holds vectors to be sent
    * downstream
    */
-
   public void addBatch(VectorContainer container) {
     setSchema(container);
     batchCount++;
@@ -116,7 +113,7 @@ public class VectorContainerAccessor implements BatchAccessor {
   public SelectionVector4 selectionVector4() {
     // Throws an exception by default because containers
     // do not support selection vectors.
-     return container.getSelectionVector4();
+    return container.getSelectionVector4();
   }
 
   @Override
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/ImplicitColumnManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/ImplicitColumnManager.java
index 24253c3..d6576b7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/ImplicitColumnManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/ImplicitColumnManager.java
@@ -77,7 +77,7 @@ public class ImplicitColumnManager implements MetadataManager, ReaderProjectionR
     private List<Path> files;
 
     /**
-     * Historically Drill will expand parition columns (dir0, dir1, ...)
+     * Historically Drill will expand partition columns (dir0, dir1, ...)
      * when the project list includes a wildcard.
      */
     protected boolean useLegacyWildcardExpansion = true;
@@ -135,7 +135,7 @@ public class ImplicitColumnManager implements MetadataManager, ReaderProjectionR
   private final List<MetadataColumn> metadataColumns = new ArrayList<>();
   private ConstantColumnLoader loader;
   private VectorContainer outputContainer;
-  private FileSystem fs;
+  private final FileSystem fs;
 
   /**
    * Constructor for {@link ImplicitColumnManager} for managing the insertion of file metadata
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 e6e3dfb..fc3b38d 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
@@ -406,7 +406,7 @@ public class ScanLevelProjection {
       }
       rootProjection = Projections.build(outputProj);
     }
-    readerProjection = ProjectionFilter.filterFor(
+    readerProjection = ProjectionFilter.providedSchemaFilter(
         rootProjection, readerSchema, errorContext);
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/file/FileDescrip.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/file/FileDescrip.java
new file mode 100644
index 0000000..bb92772
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/file/FileDescrip.java
@@ -0,0 +1,74 @@
+/*
+ * 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.v3.file;
+
+import org.apache.drill.exec.store.ColumnExplorer;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * Specify the file name and optional selection root. If the selection root
+ * is provided, then partitions are defined as the portion of the file name
+ * that is not also part of the selection root. That is, if selection root is
+ * /a/b and the file path is /a/b/c/d.csv, then dir0 is c.
+ */
+public class FileDescrip {
+
+  private final Path filePath;
+  private final String[] dirPath;
+
+  public FileDescrip(Path filePath, Path selectionRoot) {
+    this.filePath = filePath;
+
+    // If the data source is not a file, no file metadata is available.
+    if (selectionRoot == null || filePath == null) {
+      dirPath = null;
+      return;
+    }
+
+    // If the query is against a single file, selection root and file path
+    // will be identical, oddly.
+    Path rootPath = Path.getPathWithoutSchemeAndAuthority(selectionRoot);
+    Path bareFilePath = Path.getPathWithoutSchemeAndAuthority(filePath);
+    if (rootPath.equals(bareFilePath)) {
+      dirPath = null;
+      return;
+    }
+
+    dirPath = ColumnExplorer.parsePartitions(filePath, rootPath, false);
+    if (dirPath == null) {
+      throw new IllegalArgumentException(
+          String.format("Selection root of \"%s\" is not a leading path of \"%s\"",
+          selectionRoot.toString(), filePath.toString()));
+    }
+  }
+
+  public Path filePath() { return filePath; }
+
+  public String partition(int index) {
+    if (dirPath == null ||  dirPath.length <= index) {
+      return null;
+    }
+    return dirPath[index];
+  }
+
+  public int dirPathLength() {
+    return dirPath == null ? 0 : dirPath.length;
+  }
+
+  public boolean isSet() { return filePath != null; }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/AbstractSchemaTracker.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/AbstractSchemaTracker.java
new file mode 100644
index 0000000..20de4f2
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/AbstractSchemaTracker.java
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.scan.v3.schema;
+
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.MutableTupleSchema.ColumnHandle;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.ScanSchemaResolver.SchemaType;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.record.metadata.TupleSchema;
+
+/**
+ * Base class for the projection-based and defined-schema-based
+ * scan schema trackers.
+ */
+public abstract class AbstractSchemaTracker implements ScanSchemaTracker {
+
+  protected final CustomErrorContext errorContext;
+  protected final MutableTupleSchema schema = new MutableTupleSchema();
+  protected boolean isResolved;
+  private TupleMetadata outputSchema;
+  private int outputSchemaVersion;
+
+  public AbstractSchemaTracker(CustomErrorContext errorContext) {
+    this.errorContext = errorContext;
+  }
+
+  protected static void validateProjection(TupleMetadata projection, TupleMetadata schema) {
+    if (projection == null || SchemaUtils.isProjectAll(projection)) {
+      return;
+    }
+    if (schema.size() != projection.size()) {
+      throw new IllegalArgumentException("Defined schema and projection list do not match");
+    }
+    for (ColumnMetadata reqCol : projection) {
+      ColumnMetadata schemaCol = schema.metadata(reqCol.name());
+      if (schemaCol == null) {
+        throw new IllegalArgumentException(String.format(
+            "Defined schema and projection list do not match. " +
+            "`%s` in project list, but not in defined schema",
+            reqCol.name()));
+      }
+      if (schemaCol.isMap()) {
+        validateProjection(reqCol.tupleSchema(), schemaCol.tupleSchema());
+      }
+    }
+  }
+
+  @Override
+  public ProjectionType projectionType() { return schema.projectionType(); }
+
+  @Override
+  public CustomErrorContext errorContext() { return errorContext; }
+
+  @Override
+  public MutableTupleSchema internalSchema() { return schema; }
+
+  @Override
+  public boolean isResolved() { return isResolved; }
+
+  @Override
+  public int schemaVersion() { return schema.version(); }
+
+  protected void checkResolved() {
+    isResolved = schema.isResolved();
+  }
+
+  @Override
+  public TupleMetadata applyImplicitCols() {
+    checkResolved();
+    if (projectionType() == ProjectionType.SOME && allColumnsAreImplicit()) {
+      schema.setProjectionType(ProjectionType.NONE);
+    }
+    return implicitColumns();
+  }
+
+  private boolean allColumnsAreImplicit() {
+    for (ColumnHandle handle : schema.columns()) {
+      if (!handle.isImplicit()) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  private TupleMetadata implicitColumns() {
+    TupleMetadata implicitCols = new TupleSchema();
+    for (ColumnHandle handle : schema.columns()) {
+      ColumnMetadata col = handle.column();
+      if (SchemaUtils.isImplicit(col)) {
+        implicitCols.addColumn(col);
+      }
+    }
+    return implicitCols;
+  }
+
+  @Override
+  public TupleMetadata readerInputSchema() {
+    TupleMetadata readerInputSchema = new TupleSchema();
+    for (ColumnHandle handle : schema.columns()) {
+      if (!handle.isImplicit()) {
+        readerInputSchema.addColumn(handle.column());
+      }
+    }
+    return readerInputSchema;
+  }
+
+  @Override
+  public TupleMetadata missingColumns(TupleMetadata readerOutputSchema) {
+    TupleMetadata missingCols = new TupleSchema();
+    for (ColumnHandle handle : schema.columns()) {
+      if (handle.isImplicit()) {
+        continue;
+      }
+      ColumnMetadata readerCol = readerOutputSchema.metadata(handle.column().name());
+      if (readerCol == null) {
+        missingCols.addColumn(handle.column());
+      } else if (readerCol.isMap()) {
+        ColumnMetadata diff = MetadataUtils.diffMap(handle.column(), readerCol);
+        if (diff != null) {
+          missingCols.addColumn(diff);
+        }
+      }
+    }
+    return missingCols;
+  }
+
+  @Override
+  public void resolveMissingCols(TupleMetadata missingCols) {
+    new ScanSchemaResolver(schema, SchemaType.MISSING_COLS, false, errorContext)
+        .applySchema(missingCols);
+    checkResolved();
+  }
+
+  @Override
+  public TupleMetadata outputSchema() {
+    if (outputSchema == null || outputSchemaVersion < schema.version()) {
+      outputSchema = buildOutputSchema();
+    }
+    return outputSchema;
+  }
+
+  private TupleMetadata buildOutputSchema() {
+    TupleMetadata outputSchema = new TupleSchema();
+    for (ColumnHandle handle : schema.columns()) {
+      outputSchema.addColumn(handle.column());
+    }
+    return outputSchema;
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/DynamicSchemaFilter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/DynamicSchemaFilter.java
new file mode 100644
index 0000000..513a102
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/DynamicSchemaFilter.java
@@ -0,0 +1,255 @@
+/*
+ * 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.v3.schema;
+
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.MutableTupleSchema.ColumnHandle;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.ScanSchemaTracker.ProjectionType;
+import org.apache.drill.exec.physical.resultSet.impl.ProjectionFilter;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.DynamicColumn;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+
+/**
+ * Projection filter based on the scan schema which typically starts as fully
+ * dynamic, then becomes more concrete as the scan progresses. Enforces that
+ * projected columns must be consistent with either projection, or the existing
+ * concrete schema for that columns.
+ */
+public abstract class DynamicSchemaFilter implements ProjectionFilter {
+
+  /**
+   * Describes how to handle candidate columns not currently in the
+   * scan schema, which turns out to be a surprisingly complex
+   * question. At the top level, we add columns only if the query
+   * contains a wildcard. But, within maps, there are additional
+   * constraints: we can add new members to a map even if the query
+   * itself does not contain a wildcard.
+   */
+  public enum NewColumnsMode {
+
+    /**
+     * No new columns are allowed at this level or in maps
+     * below this level. Occurs when the schema is defined
+     * or with a strict provided schema.
+     */
+    NONE,
+
+    /**
+     * New columns are allowed at this level and below.
+     * Occurs in a wildcard projection in which there are no
+     * constraints on the columns which can be added.
+     */
+    ALL,
+
+    /**
+     * New columns cannot be added at this level, but can be
+     * added in maps below this level. Occurs in a query where
+     * the projection list is explicit: {@code a, b, m}, and it
+     * turns out that {@code m} is a map. A simple {@code m}
+     * projection is logically equivalent to {@code m.*}.
+     * <p>
+     * This same logic can apply to maps if the project list contains
+     * something like {@code m.a, m.m2}, and {@code m2} turns out
+     * to be a map.
+     */
+    CHILD_ONLY
+  }
+
+  protected final CustomErrorContext errorContext;
+  protected final String source;
+  protected final NewColumnsMode newColumnsMode;
+
+  public DynamicSchemaFilter(CustomErrorContext errorContext,
+      String source, NewColumnsMode newColumnsMode) {
+    this.errorContext = errorContext;
+    this.source = source;
+    this.newColumnsMode = newColumnsMode;
+  }
+
+  public ProjResult buildProjection(ColumnMetadata schemaCol, ColumnMetadata probeCol) {
+    if (schemaCol == null) {
+      return newColumnProjection();
+    }
+    if (schemaCol instanceof ProjectedColumn) {
+
+      // Column comes from the project list
+      return fromProjection((ProjectedColumn) schemaCol, probeCol);
+    } else if (schemaCol instanceof DynamicColumn) {
+      return PROJECTED;
+    } else {
+
+      // Column has a schema defined earlier.
+      return fromSchema(schemaCol, probeCol);
+    }
+  }
+
+  protected ProjResult newColumnProjection() {
+    // No match. If this is an open schema, project the column
+    // and its children, if any. If closed, don't project the column.
+    return newColumnsMode == NewColumnsMode.ALL ? PROJECTED : NOT_PROJECTED;
+  }
+
+  /**
+   * A column exists in the scan schema, and is dynamic. The proposed
+   * column can be projected. First, however, we verify consistency.
+   */
+  private ProjResult fromProjection(ProjectedColumn projCol, ColumnMetadata probeCol) {
+
+    // Verify that the reader/provided column is consistent with projection
+    SchemaUtils.verifyCompatibility(projCol, probeCol, source, errorContext);
+
+    if (projCol.isMap()) {
+
+      // The projected column is a map (has named members). Track these to
+      // project children.
+      return new ProjResult(true, projCol, mapProjection(projCol));
+    } else {
+
+      // The projected column is generic. Harmlessly project all children
+      // for both map and non-map columns.
+      return new ProjResult(true, projCol, PROJECT_ALL);
+    }
+  }
+
+  /**
+   * A column exists in the scan schema, and is concrete. The proposed
+   * column can be projected. Verify consistency. The reader should not be
+   * proposing a column with the wrong type or mode since it was told the
+   * reader input schema, and that schema was derived from a provided schema
+   * (which should be acceptable to the reader) or by a prior reader in the
+   * same scan.
+   */
+  protected ProjResult fromSchema(ColumnMetadata schemaCol,
+      ColumnMetadata probeCol) {
+    SchemaUtils.verifyConsistency(schemaCol, probeCol, source, errorContext);
+    if (schemaCol.isMap()) {
+      return new ProjResult(true, schemaCol, mapProjection(schemaCol));
+    } else {
+      return new ProjResult(true, schemaCol);
+    }
+  }
+
+  private ProjectionFilter mapProjection(ColumnMetadata map) {
+    return new DynamicTupleFilter(map.tupleSchema(),
+        newColumnsMode != NewColumnsMode.NONE,
+        errorContext, source);
+  }
+
+  @Override
+  public boolean isProjected(String colName) {
+
+    // To avoid duplicating logic, create a dynamic column
+    // to run though the above checks.
+    return projection(MetadataUtils.newDynamic(colName)).isProjected;
+  }
+
+  /**
+   * Filter for a map, represented by a {@code TupleMetadata}.
+   */
+  public static class DynamicTupleFilter extends DynamicSchemaFilter {
+    private final TupleMetadata mapSchema;
+
+    public DynamicTupleFilter(TupleMetadata mapSchema, boolean isOpen,
+        CustomErrorContext errorContext,
+        String source) {
+      super(errorContext, source, newColumnsMode(mapSchema, isOpen));
+      this.mapSchema = mapSchema;
+    }
+
+    private static NewColumnsMode newColumnsMode(TupleMetadata projectionSet, boolean isOpen) {
+      if (!isOpen) {
+        return NewColumnsMode.NONE;
+      } else if (SchemaUtils.isProjectAll(projectionSet)) {
+        return NewColumnsMode.ALL;
+      } else {
+        return NewColumnsMode.CHILD_ONLY;
+      }
+    }
+
+    public static ProjectionFilter filterFor(DynamicColumn col, boolean allowMapAdditions,
+        CustomErrorContext errorContext, String source) {
+      if (col.isMap()) {
+        return new DynamicTupleFilter(col.tupleSchema(), allowMapAdditions, errorContext, source);
+      } else {
+        return PROJECT_ALL;
+      }
+    }
+
+    public DynamicTupleFilter(TupleMetadata projectionSet, CustomErrorContext errorContext) {
+      this(projectionSet, true, errorContext, "Reader");
+    }
+
+    @Override
+    public ProjResult projection(ColumnMetadata col) {
+      return buildProjection(mapSchema.metadata(col.name()), col);
+    }
+
+    @Override
+    public boolean isEmpty() {
+      return mapSchema.isEmpty();
+    }
+  }
+
+  /**
+   * Filter for the top-level dynamic schema.
+   */
+  public static class RowSchemaFilter extends DynamicSchemaFilter {
+    private final MutableTupleSchema schema;
+
+    public RowSchemaFilter(MutableTupleSchema schema, boolean allowMapChanges,
+        CustomErrorContext errorContext) {
+      super(errorContext, "Reader", newColumnsMode(schema, allowMapChanges));
+      this.schema = schema;
+    }
+
+    private static NewColumnsMode newColumnsMode(MutableTupleSchema schema, boolean allowMapChanges) {
+      if (schema.projectionType() == ProjectionType.ALL) {
+        return NewColumnsMode.ALL;
+      } else if (allowMapChanges) {
+        return NewColumnsMode.CHILD_ONLY;
+      } else {
+        return NewColumnsMode.NONE;
+      }
+    }
+
+    @Override
+    public ProjResult projection(ColumnMetadata col) {
+      ColumnHandle handle = schema.find(col.name());
+      if (handle == null) {
+        return newColumnProjection();
+      }
+
+      // Top-level columns can be implicit. Do not project a reader
+      // column of the same name as an implicit column, even if this
+      // is a wildcard projection.
+      if (handle.isImplicit()) {
+        logger.warn("Ignoring reader column with the same name as an implicit column: {}",
+            col.name());
+        return NOT_PROJECTED;
+      }
+      return buildProjection(handle.column(), col);
+    }
+
+    @Override
+    public boolean isEmpty() {
+      return schema.projectionType() == ScanSchemaTracker.ProjectionType.NONE;
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/ImplicitColumnResolver.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/ImplicitColumnResolver.java
new file mode 100644
index 0000000..fcc3ed8
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/ImplicitColumnResolver.java
@@ -0,0 +1,410 @@
+/*
+ * 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.v3.schema;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.map.CaseInsensitiveMap;
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.physical.impl.scan.v3.file.FileDescrip;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.MutableTupleSchema.ColumnHandle;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.server.options.OptionSet;
+import org.apache.drill.exec.store.ColumnExplorer.ImplicitFileColumns;
+import org.apache.drill.shaded.guava.com.google.common.base.Strings;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Manages the resolution of implicit file metadata and partition columns.
+ * Parses the file metadata columns from the projection list. Creates a parse
+ * result which drives loading data into vectors. Supports renaming the columns
+ * via session options.
+ * <p>
+ * Lifecycle:
+ * <ul>
+ * <li>At the start of the scan, the parser looks for implicit and partition
+ * columns in the scan schema, resolving matching columns to be implicit
+ * columns, and building up a description of those columns for use later.</li>
+ * <li>If the projection list contains a wildcard, it can also contain implicit
+ * columns. If running in legacy mode, inserts partition columns when the query
+ * contains a wildcard.</li>
+ * <li>On each file (on each reader), the parse result allows generating actual
+ * values for each column, which are then written into the corresponding value
+ * vectors.</li>
+ * </ul>
+ * <p>
+ * Assumes that the partition count is fixed at runtime; that it is determined
+ * at plan time and provided in the plan. This ensures that the schema is stable
+ * across readers: even a reader at the top-most partition will produce columns
+ * for all partitions if using legacy mode wildcard expansion.
+ */
+public class ImplicitColumnResolver {
+  private static final Logger logger = LoggerFactory.getLogger(ImplicitColumnResolver.class);
+
+  public static class ImplicitColumnOptions {
+
+    protected OptionSet optionSet;
+
+    protected int maxPartitionDepth;
+
+    /**
+     * Historically Drill will expand partition columns (dir0, dir1, ...)
+     * when the project list includes a wildcard.
+     */
+    protected boolean useLegacyWildcardExpansion = true;
+
+    public ImplicitColumnOptions optionSet(OptionSet optionSet) {
+      this.optionSet = optionSet;
+      return this;
+   }
+
+   /**
+    * The maximum partition depth for any file in this query. Specifies
+    * the maximum number of {@code diri} columns that this parser will
+    * recognize or generate.
+    */
+   public ImplicitColumnOptions maxPartitionDepth(int maxPartitionDepth) {
+     this.maxPartitionDepth = maxPartitionDepth;
+     return this;
+   }
+
+    /**
+     * 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 ImplicitColumnOptions useLegacyWildcardExpansion(boolean flag) {
+      useLegacyWildcardExpansion = flag;
+      return this;
+    }
+  }
+
+  /**
+   * Provides a function to resolve an implicit column given a description
+   * of the input file.
+   */
+  public interface ColumnMarker {
+    String resolve(FileDescrip fileInfo);
+  }
+
+  /**
+   * Implicit column defined by an {@link ImplicitFileColumns}.
+   */
+  public static class ImplicitColumnMarker implements ColumnMarker {
+    private final ImplicitFileColumns defn;
+
+    public ImplicitColumnMarker(ImplicitFileColumns defn) {
+      this.defn = defn;
+    }
+
+    @Override
+    public String resolve(FileDescrip fileInfo) {
+      return defn.getValue(fileInfo.filePath());
+    }
+  }
+
+  /**
+   * Partition column defined by a partition depth from the scan
+   * root folder. Partitions that reference non-existent directory levels
+   * are null.
+   */
+  public static class PartitionColumnMarker implements ColumnMarker {
+    private final int partition;
+
+    private PartitionColumnMarker(int partition) {
+      this.partition = partition;
+    }
+
+    @Override
+    public String resolve(FileDescrip fileInfo) {
+      return fileInfo.partition(partition);
+    }
+  }
+
+  /**
+   * The result of scanning the scan output schema to identify implicit and
+   * partition columns. Defines a sub-schema of just these columns, along with
+   * column markers which resolve the columns for each file.
+   */
+  public static class ParseResult {
+    private final List<ColumnMarker> columns;
+    private final TupleMetadata schema;
+
+    protected ParseResult(List<ColumnMarker> columns, TupleMetadata schema) {
+      this.columns = columns;
+      this.schema = schema;
+    }
+
+    public TupleMetadata schema() { return schema; }
+    public List<ColumnMarker> columns() { return columns; }
+
+    public Object[] resolve(FileDescrip fileInfo) {
+      Object values[] = new Object[columns.size()];
+      for (int i = 0; i < values.length; i++) {
+        values[i] = columns.get(i).resolve(fileInfo);
+      }
+      return values;
+    }
+  }
+
+  private static class ImplicitColumnParser {
+    private final ImplicitColumnResolver parser;
+    private final ScanSchemaTracker tracker;
+    private final MutableTupleSchema scanSchema;
+    private final List<ColumnMarker> columns = new ArrayList<>();
+    private final Set<Integer> referencedPartitions = new HashSet<>();
+
+    protected ImplicitColumnParser(ImplicitColumnResolver parser, ScanSchemaTracker tracker) {
+      this.parser = parser;
+      this.tracker = tracker;
+      this.scanSchema = tracker.internalSchema();
+    }
+
+    protected ParseResult parse() {
+      for (ColumnHandle col : tracker.internalSchema().columns()) {
+        matchColumn(parser, col);
+      }
+      if (tracker.internalSchema().projectionType() == ScanSchemaTracker.ProjectionType.ALL) {
+        expandWildcard();
+      }
+
+      // Have the tracker gather the implicit columns so they appear
+      // in the same order as the output schema, even if a wildcard
+      // appears out-of-order:
+      // SELECT *, fileName
+      // SELECT fileName, *
+      return new ParseResult(columns, tracker.applyImplicitCols());
+    }
+
+    private void expandWildcard() {
+      if (!parser.useLegacyWildcardExpansion) {
+        return;
+      }
+
+      // Legacy wildcard expansion: include the file partitions for this file.
+      // This is a disadvantage for a * query: files at different directory
+      // levels will have different numbers of columns. Would be better to
+      // return this data as an array at some point.
+      // Append this after the *, keeping the * for later expansion.
+      for (int i = 0; i < parser.maxPartitionDepth; i++) {
+        if (referencedPartitions.contains(i)) {
+          continue;
+        }
+        ColumnMetadata resolved = MetadataUtils.newScalar(parser.partitionName(i), PARTITION_COL_TYPE);
+        SchemaUtils.markAsPartition(resolved, i);
+        columns.add(new PartitionColumnMarker(i));
+        tracker.expandImplicitCol(resolved);
+        referencedPartitions.add(i);
+      }
+    }
+
+    private void matchColumn(ImplicitColumnResolver parser, ColumnHandle col) {
+      String colType = SchemaUtils.implicitColType(col.column());
+      if (colType != null) {
+        resolveTaggedColumn(parser, col, colType);
+        return;
+      } else if (col.column().isDynamic()) {
+        matchByName(parser, col);
+      }
+    }
+
+    private void resolveTaggedColumn(ImplicitColumnResolver parser,
+        ColumnHandle col, String colType) {
+      Matcher m = parser.partitionTypePattern.matcher(colType);
+      if (m.matches()) {
+        resolvePartitionColumn(m, parser, col);
+        return;
+      }
+
+      ImplicitFileColumns defn = parser.typeDefs.get(colType);
+      if (defn != null) {
+        resolveImplicitColumn(defn, parser, col);
+        return;
+      }
+      resolveUnknownColumn(parser, col, colType);
+    }
+
+    private void resolvePartitionColumn(Matcher m, ImplicitColumnResolver parser, ColumnHandle col) {
+
+      // The provided schema column must be of the correct type and mode.
+      ColumnMetadata colSchema = col.column();
+      if (colSchema.type() != MinorType.VARCHAR ||
+          colSchema.mode() != DataMode.OPTIONAL) {
+        throw UserException.validationError()
+            .message("Provided column %s is marked as a parition column, but is of the wrong type",
+                colSchema.columnString())
+            .addContext("Expected type", MinorType.VARCHAR.name())
+            .addContext("Expected cardinality", DataMode.OPTIONAL.name())
+            .addContext(parser.errorContext)
+            .build(logger);
+      }
+
+      // Partition column
+      int partitionIndex = Integer.parseInt(m.group(1));
+      columns.add(new PartitionColumnMarker(partitionIndex));
+      col.markImplicit();
+
+      // Remember the partition for later wildcard expansion
+      referencedPartitions.add(partitionIndex);
+    }
+
+    private void resolveImplicitColumn(ImplicitFileColumns defn,
+        ImplicitColumnResolver parser, ColumnHandle col) {
+
+      // The provided schema column must be of the correct type and mode.
+      ColumnMetadata colSchema = col.column();
+      if (colSchema.type() != MinorType.VARCHAR ||
+          colSchema.mode() == DataMode.REPEATED) {
+        throw UserException.validationError()
+            .message("Provided column %s is marked as an implicit column '%s', but is of the wrong type",
+                colSchema.columnString(), defn.propertyValue())
+            .addContext("Expected type", MinorType.VARCHAR.name())
+            .addContext("Expected cardinality", String.format("%s or %s",
+                DataMode.REQUIRED.name(), DataMode.OPTIONAL.name()))
+            .addContext(parser.errorContext)
+            .build(logger);
+      }
+      columns.add(new ImplicitColumnMarker(defn));
+      col.markImplicit();
+    }
+
+    private void resolveUnknownColumn(ImplicitColumnResolver parser,
+        ColumnHandle col, String colType) {
+      throw UserException.validationError()
+          .message("Provided column %s references an undefined implicit column type '%s'",
+              col.column().columnString(), colType)
+          .addContext("Expected type", MinorType.VARCHAR.name())
+          .addContext("Expected cardinality", String.format("%s or %s",
+              DataMode.REQUIRED.name(), DataMode.OPTIONAL.name()))
+          .addContext(parser.errorContext)
+          .build(logger);
+    }
+
+    private void matchByName(ImplicitColumnResolver parser, ColumnHandle col) {
+      Matcher m = parser.partitionPattern.matcher(col.name());
+      if (m.matches()) {
+        buildPartitionColumn(m, parser, col);
+        return;
+      }
+
+      ImplicitFileColumns defn = parser.colDefs.get(col.name());
+      if (defn != null) {
+        buildImplicitColumn(defn, parser, col);
+      }
+    }
+
+    private void buildPartitionColumn(Matcher m, ImplicitColumnResolver parser, ColumnHandle col) {
+
+      // If the projected column is a map or array, then it shadows the
+      // partition column. Example: dir0.x, dir0[2].
+      ProjectedColumn projCol = (ProjectedColumn) col.column();
+      if (!projCol.isSimple()) {
+        logger.warn("Projected column {} shadows partition column {}",
+            projCol.projectString(), col.name());
+        return;
+      }
+
+      // Partition column
+      int partitionIndex = Integer.parseInt(m.group(1));
+      ColumnMetadata resolved = MetadataUtils.newScalar(col.name(), PARTITION_COL_TYPE);
+      SchemaUtils.markAsPartition(resolved, partitionIndex);
+      columns.add(new PartitionColumnMarker(partitionIndex));
+      scanSchema.resolveImplicit(col, resolved);
+
+      // Remember the partition for later wildcard expansion
+      referencedPartitions.add(partitionIndex);
+    }
+
+    private void buildImplicitColumn(ImplicitFileColumns defn,
+        ImplicitColumnResolver parser, ColumnHandle col) {
+
+      // If the projected column is a map or array, then it shadows the
+      // metadata column. Example: filename.x, filename[2].
+      ProjectedColumn projCol = (ProjectedColumn) col.column();
+      if (!projCol.isSimple()) {
+        logger.warn("Projected column {} shadows implicit column {}",
+            projCol.projectString(), col.name());
+      } else {
+        ColumnMetadata resolved = MetadataUtils.newScalar(col.name(), IMPLICIT_COL_TYPE);
+        SchemaUtils.markImplicit(resolved, defn.propertyValue());
+        columns.add(new ImplicitColumnMarker(defn));
+        scanSchema.resolveImplicit(col, resolved);
+      }
+    }
+  }
+
+  public static final MajorType IMPLICIT_COL_TYPE = Types.required(MinorType.VARCHAR);
+  public static final MajorType PARTITION_COL_TYPE =  Types.optional(MinorType.VARCHAR);
+
+  private final int maxPartitionDepth;
+  private final boolean useLegacyWildcardExpansion;
+  private final String partitionDesignator;
+  private final Pattern partitionPattern;
+  private final Pattern partitionTypePattern;
+  private final Map<String, ImplicitFileColumns> colDefs = CaseInsensitiveMap.newHashMap();
+  private final Map<String, ImplicitFileColumns> typeDefs = CaseInsensitiveMap.newHashMap();
+  private final CustomErrorContext errorContext;
+
+  public ImplicitColumnResolver(ImplicitColumnOptions options, CustomErrorContext errorContext) {
+    this.errorContext = errorContext;
+    this.maxPartitionDepth = options.maxPartitionDepth;
+    this.useLegacyWildcardExpansion = options.useLegacyWildcardExpansion;
+    this.partitionDesignator = options.optionSet.getString(ExecConstants.FILESYSTEM_PARTITION_COLUMN_LABEL);
+    this.partitionPattern = Pattern.compile(partitionDesignator + "(\\d+)", Pattern.CASE_INSENSITIVE);
+    if (partitionDesignator.equals(ColumnMetadata.IMPLICIT_PARTITION_PREFIX)) {
+      this.partitionTypePattern = partitionPattern;
+    } else {
+      this.partitionTypePattern = Pattern.compile(ColumnMetadata.IMPLICIT_PARTITION_PREFIX + "(\\d+)",
+          Pattern.CASE_INSENSITIVE);
+    }
+    for (ImplicitFileColumns defn : ImplicitFileColumns.values()) {
+      String colName = options.optionSet.getString(defn.optionName());
+      if (!Strings.isNullOrEmpty(colName)) {
+        this.colDefs.put(colName, defn);
+      }
+      typeDefs.put(defn.propertyValue(), defn);
+    }
+  }
+
+  public ParseResult parse(ScanSchemaTracker tracker) {
+    return new ImplicitColumnParser(this, tracker).parse();
+  }
+
+  public String partitionName(int partition) {
+    return partitionDesignator + partition;
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/MutableTupleMetadata.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/MutableTupleMetadata.java
new file mode 100644
index 0000000..d5f46d2
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/MutableTupleMetadata.java
@@ -0,0 +1,220 @@
+/*
+ * 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.v3.schema;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.drill.common.map.CaseInsensitiveMap;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.ScanSchemaTracker.ProjectionType;
+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.base.Preconditions;
+
+/**
+ * A mutable form of a tuple schema. Allows insertions (at the wildcard position),
+ * and replacing columns (as the schema becomes resolved). Tracks implicit columns
+ * (those not filled in by the reader).
+ * <p>
+ * Does not implement the {@code TupleMetadata} interface because that interface
+ * has far more functionality than is needed here, and assumes that column order
+ * remains fixed (and hence columns can be addressed by position) which is not
+ * true for this class.
+ * <p>
+ * This class represents the top-level tuple (the row.) Maps are also dynamic,
+ * but provide a subset of resolution options:
+ * map fields cannot be implicit. They can, however, be defined,
+ * provided, discovered or missing. Map columns can start unresolved
+ * if the map comes from projection. A map itself can be resolved,
+ * but its members may be unresolved. New map members may only be added at the
+ * end (there is no equivalent of a wildcard position.)
+ */
+public class MutableTupleMetadata {
+
+  /**
+   * Holder for a column to allow inserting and replacing columns within
+   * the top-level project list. Changes to the column within the holder
+   * must go through the tuple itself so we can track schema versions.
+   * <p>
+   * Tracks the resolution status of each individual column as
+   * described for {@link ScanSchemaTracker}. Models a column throughout the
+   * projection lifecycle. Columns evolve from unresolved to resolved at
+   * different times. Columns are either implicit (defined by the framework)
+   * or normal (defined by the reader). Columns can be defined by the
+   * planner (via a defined schema), partially defined (via a provided
+   * schema), or discovered by the reader. Regardless of the path
+   * to definition, by the time the first batch is delivered downstream,
+   * each column has an output schema which describes the data.
+   */
+  public static class ColumnHandle {
+    private ColumnMetadata col;
+    private boolean isImplicit;
+
+    public ColumnHandle(ColumnMetadata col) {
+      this.col = col;
+      this.isImplicit = SchemaUtils.isImplicit(col);
+    }
+
+    public String name() {
+      return col.name();
+    }
+
+    private void replace(ColumnMetadata col) {
+      this.col = col;
+    }
+
+    private void resolve(ColumnMetadata col) {
+      SchemaUtils.mergeColProperties(this.col, col);
+      this.col = col;
+    }
+
+    private void resolveImplicit(ColumnMetadata col) {
+      SchemaUtils.mergeColProperties(this.col, col);
+      this.col = col;
+      markImplicit();
+    }
+
+    public void markImplicit() {
+      Preconditions.checkState(SchemaUtils.isImplicit(col));
+      isImplicit = true;
+    }
+
+    public ColumnMetadata column() { return col; }
+    public boolean isImplicit() { return isImplicit; }
+
+    @Override
+    public String toString() {
+      return col.toString();
+    }
+  }
+
+  protected final List<MutableTupleMetadata.ColumnHandle> columns = new ArrayList<>();
+  protected final Map<String, MutableTupleMetadata.ColumnHandle> nameIndex =
+      CaseInsensitiveMap.newHashMap();
+  private ProjectionType projType;
+  private int insertPoint = -1;
+  private int version;
+
+  public void setProjectionType(ScanSchemaTracker.ProjectionType type) {
+    this.projType = type;
+  }
+
+  public void setInsertPoint(int insertPoint) {
+    Preconditions.checkArgument(insertPoint == -1 ||
+        insertPoint >= 0 && insertPoint <= size());
+    this.insertPoint = insertPoint;
+  }
+
+  public ScanSchemaTracker.ProjectionType projectionType() { return projType; }
+  public int size() { return columns.size(); }
+  public int version() { return version; }
+
+  /**
+   * Provide the list of partially-resolved columns. Primarily for
+   * the implicit column parser.
+   */
+  public List<MutableTupleMetadata.ColumnHandle> columns() { return columns; }
+
+  public MutableTupleMetadata.ColumnHandle find(String colName) {
+    return nameIndex.get(colName);
+  }
+
+  public void copyFrom(TupleMetadata from) {
+    if (from.isEmpty()) {
+      return;
+    }
+    for (ColumnMetadata projCol : from) {
+      add(projCol.copy());
+    }
+    version++;
+  }
+
+  public void add(ColumnMetadata col) {
+    MutableTupleMetadata.ColumnHandle holder = new ColumnHandle(col);
+    columns.add(holder);
+    addIndex(holder);
+    version++;
+  }
+
+  public void addIndex(MutableTupleMetadata.ColumnHandle holder) {
+    if (nameIndex.put(holder.column().name(), holder) != null) {
+      throw new IllegalArgumentException("Duplicate scan projection column: " + holder.name());
+    }
+  }
+
+  public void insert(int posn, ColumnMetadata col) {
+    MutableTupleMetadata.ColumnHandle holder = new ColumnHandle(col);
+    columns.add(posn, holder);
+    addIndex(holder);
+    version++;
+  }
+
+  public void insert(ColumnMetadata col) {
+    insert(insertPoint++, col);
+  }
+
+  public boolean isResolved() {
+    for (MutableTupleMetadata.ColumnHandle handle : columns) {
+      if (!isColumnResolved(handle.column())) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  private boolean isColumnResolved(ColumnMetadata col) {
+    return !col.isDynamic() && (!col.isMap() || isMapResolved(col.tupleSchema()));
+  }
+
+  private boolean isMapResolved(TupleMetadata mapSchema) {
+    for (ColumnMetadata col : mapSchema) {
+      if (col.isDynamic()) {
+        return false;
+      }
+      if (col.isMap() && !isMapResolved(col.tupleSchema())) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  public TupleMetadata toSchema() {
+    TupleMetadata schema = new TupleSchema();
+    for (MutableTupleMetadata.ColumnHandle col : columns) {
+      schema.addColumn(col.column());
+    }
+    return schema;
+  }
+
+  public void resolveImplicit(ColumnHandle col, ColumnMetadata resolved) {
+    col.resolveImplicit(resolved);
+    version++;
+  }
+
+  public void replace(ColumnHandle col, ColumnMetadata resolved) {
+    col.replace(resolved);
+    version++;
+  }
+
+  public void resolve(ColumnHandle col, ColumnMetadata resolved) {
+    col.resolve(resolved);
+    version++;
+  }
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/MutableTupleSchema.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/MutableTupleSchema.java
new file mode 100644
index 0000000..ba3483f
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/MutableTupleSchema.java
@@ -0,0 +1,220 @@
+/*
+ * 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.v3.schema;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.drill.common.map.CaseInsensitiveMap;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.ScanSchemaTracker.ProjectionType;
+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.base.Preconditions;
+
+/**
+ * A mutable form of a tuple schema. Allows insertions (at the wildcard position),
+ * and replacing columns (as the schema becomes resolved). Tracks implicit columns
+ * (those not filled in by the reader).
+ * <p>
+ * Does not implement the {@code TupleMetadata} interface because that interface
+ * has far more functionality than is needed here, and assumes that column order
+ * remains fixed (and hence columns can be addressed by position) which is not
+ * true for this class.
+ * <p>
+ * This class represents the top-level tuple (the row.) Maps are also dynamic,
+ * but provide a subset of resolution options:
+ * map fields cannot be implicit. They can, however, be defined,
+ * provided, discovered or missing. Map columns can start unresolved
+ * if the map comes from projection. A map itself can be resolved,
+ * but its members may be unresolved. New map members may only be added at the
+ * end (there is no equivalent of a wildcard position.)
+ */
+public class MutableTupleSchema {
+
+  /**
+   * Holder for a column to allow inserting and replacing columns within
+   * the top-level project list. Changes to the column within the holder
+   * must go through the tuple itself so we can track schema versions.
+   * <p>
+   * Tracks the resolution status of each individual column as
+   * described for {@link ScanSchemaTracker}. Models a column throughout the
+   * projection lifecycle. Columns evolve from unresolved to resolved at
+   * different times. Columns are either implicit (defined by the framework)
+   * or normal (defined by the reader). Columns can be defined by the
+   * planner (via a defined schema), partially defined (via a provided
+   * schema), or discovered by the reader. Regardless of the path
+   * to definition, by the time the first batch is delivered downstream,
+   * each column has an output schema which describes the data.
+   */
+  public static class ColumnHandle {
+    private ColumnMetadata col;
+    private boolean isImplicit;
+
+    public ColumnHandle(ColumnMetadata col) {
+      this.col = col;
+      this.isImplicit = SchemaUtils.isImplicit(col);
+    }
+
+    public String name() {
+      return col.name();
+    }
+
+    private void replace(ColumnMetadata col) {
+      this.col = col;
+    }
+
+    private void resolve(ColumnMetadata col) {
+      SchemaUtils.mergeColProperties(this.col, col);
+      this.col = col;
+    }
+
+    private void resolveImplicit(ColumnMetadata col) {
+      SchemaUtils.mergeColProperties(this.col, col);
+      this.col = col;
+      markImplicit();
+    }
+
+    public void markImplicit() {
+      Preconditions.checkState(SchemaUtils.isImplicit(col));
+      isImplicit = true;
+    }
+
+    public ColumnMetadata column() { return col; }
+    public boolean isImplicit() { return isImplicit; }
+
+    @Override
+    public String toString() {
+      return col.toString();
+    }
+  }
+
+  protected final List<MutableTupleSchema.ColumnHandle> columns = new ArrayList<>();
+  protected final Map<String, MutableTupleSchema.ColumnHandle> nameIndex =
+      CaseInsensitiveMap.newHashMap();
+  private ProjectionType projType;
+  private int insertPoint = -1;
+  private int version;
+
+  public void setProjectionType(ScanSchemaTracker.ProjectionType type) {
+    this.projType = type;
+  }
+
+  public void setInsertPoint(int insertPoint) {
+    Preconditions.checkArgument(insertPoint == -1 ||
+        insertPoint >= 0 && insertPoint <= size());
+    this.insertPoint = insertPoint;
+  }
+
+  public ScanSchemaTracker.ProjectionType projectionType() { return projType; }
+  public int size() { return columns.size(); }
+  public int version() { return version; }
+
+  /**
+   * Provide the list of partially-resolved columns. Primarily for
+   * the implicit column parser.
+   */
+  public List<MutableTupleSchema.ColumnHandle> columns() { return columns; }
+
+  public MutableTupleSchema.ColumnHandle find(String colName) {
+    return nameIndex.get(colName);
+  }
+
+  public void copyFrom(TupleMetadata from) {
+    if (from.isEmpty()) {
+      return;
+    }
+    for (ColumnMetadata projCol : from) {
+      add(projCol.copy());
+    }
+    version++;
+  }
+
+  public void add(ColumnMetadata col) {
+    MutableTupleSchema.ColumnHandle holder = new ColumnHandle(col);
+    columns.add(holder);
+    addIndex(holder);
+    version++;
+  }
+
+  public void addIndex(MutableTupleSchema.ColumnHandle holder) {
+    if (nameIndex.put(holder.column().name(), holder) != null) {
+      throw new IllegalArgumentException("Duplicate scan projection column: " + holder.name());
+    }
+  }
+
+  public void insert(int posn, ColumnMetadata col) {
+    MutableTupleSchema.ColumnHandle holder = new ColumnHandle(col);
+    columns.add(posn, holder);
+    addIndex(holder);
+    version++;
+  }
+
+  public void insert(ColumnMetadata col) {
+    insert(insertPoint++, col);
+  }
+
+  public boolean isResolved() {
+    for (MutableTupleSchema.ColumnHandle handle : columns) {
+      if (!isColumnResolved(handle.column())) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  private boolean isColumnResolved(ColumnMetadata col) {
+    return !col.isDynamic() && (!col.isMap() || isMapResolved(col.tupleSchema()));
+  }
+
+  private boolean isMapResolved(TupleMetadata mapSchema) {
+    for (ColumnMetadata col : mapSchema) {
+      if (col.isDynamic()) {
+        return false;
+      }
+      if (col.isMap() && !isMapResolved(col.tupleSchema())) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  public TupleMetadata toSchema() {
+    TupleMetadata schema = new TupleSchema();
+    for (MutableTupleSchema.ColumnHandle col : columns) {
+      schema.addColumn(col.column());
+    }
+    return schema;
+  }
+
+  public void resolveImplicit(ColumnHandle col, ColumnMetadata resolved) {
+    col.resolveImplicit(resolved);
+    version++;
+  }
+
+  public void replace(ColumnHandle col, ColumnMetadata resolved) {
+    col.replace(resolved);
+    version++;
+  }
+
+  public void resolve(ColumnHandle col, ColumnMetadata resolved) {
+    col.resolve(resolved);
+    version++;
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/ProjectedColumn.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/ProjectedColumn.java
new file mode 100644
index 0000000..4d9abce
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/ProjectedColumn.java
@@ -0,0 +1,190 @@
+/*
+ * 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.v3.schema;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.DynamicColumn;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.record.metadata.TupleSchema;
+
+/**
+ * Enhanced form of a dynamic column which records all information from
+ * the project list.
+ */
+public class ProjectedColumn extends DynamicColumn {
+
+  /**
+   * Marker to indicate that that a) the item is an
+   * array, and b) that all indexes are to be projected.
+   * Used when seeing both a and a[x].
+   */
+  private static final Set<Integer> ALL_INDEXES = new HashSet<>();
+
+  private int refCount = 1;
+  private int arrayDims;
+  private Set<Integer> indexes;
+  private TupleMetadata members;
+
+  public ProjectedColumn(String name) {
+    super(name);
+  }
+
+  protected void bumpRefCount() { refCount++; }
+
+  public int refCount() { return refCount; }
+
+  public boolean isSimple() {
+    return !isArray() && !isMap();
+  }
+
+  @Override
+  public boolean isMap() {
+    return members != null;
+  }
+
+  public void projectAllElements() {
+    indexes = ALL_INDEXES;
+  }
+
+  public void becomeArray(int dims) {
+    arrayDims = dims;
+    indexes = indexes == null ? new HashSet<>() : indexes;
+  }
+
+  public int arrayDims() { return arrayDims; }
+
+  @Override
+  public boolean isArray() {
+    return arrayDims > 0;
+  }
+
+  protected void addIndex(int index) {
+    if (indexes == null) {
+      indexes = new HashSet<>();
+    }
+    if (indexes != ALL_INDEXES) {
+      indexes.add(index);
+    }
+  }
+
+  public boolean hasIndexes() {
+    return isArray() && indexes != ALL_INDEXES;
+  }
+
+  public boolean hasIndex(int index) {
+    return hasIndexes() && indexes.contains(index);
+  }
+
+  public int maxIndex() {
+    if (!hasIndexes()) {
+      return 0;
+    }
+    int max = 0;
+    for (final Integer index : indexes) {
+      max = Math.max(max, index);
+    }
+    return max;
+  }
+
+  public boolean[] indexes() {
+    if (!hasIndexes()) {
+      return null;
+    }
+    final int max = maxIndex();
+    final boolean map[] = new boolean[max+1];
+    for (final Integer index : indexes) {
+      map[index] = true;
+    }
+    return map;
+  }
+
+  public void projectAllMembers() {
+    if (members == null) {
+      members = new TupleSchema();
+    }
+    members.setProperty(ScanProjectionParser.PROJECTION_TYPE_PROP, ScanProjectionParser.PROJECT_ALL);
+  }
+
+  public TupleMetadata explicitMembers() {
+    if (members == null) {
+      members = new TupleSchema();
+    }
+    return members;
+  }
+
+  @Override
+  public TupleMetadata tupleSchema() { return members; }
+
+  @Override
+  protected void appendContents(StringBuilder buf) {
+    appendArray(buf);
+    if (isMap()) {
+      buf.append(" members=").append(members.toString());
+    }
+  }
+
+  private void appendArray(StringBuilder buf) {
+    if (isArray()) {
+      buf.append("[");
+      if (indexes == ALL_INDEXES) {
+        buf.append("*");
+      } else {
+        List<String> idxs = indexes.stream().sorted().map(i -> Integer.toString(i)).collect(Collectors.toList());
+        buf.append(String.join(", ", idxs));
+      }
+      buf.append("]");
+    }
+  }
+
+  @Override
+  public ColumnMetadata copy() {
+    ProjectedColumn copy = new ProjectedColumn(name);
+    copy.refCount = refCount;
+    copy.arrayDims = arrayDims;
+    copy.indexes = indexes; // Indexes are immutable after parsing
+    copy.members = members == null ? null : members.copy();
+    return copy;
+  }
+
+  public String projectString() {
+    StringBuilder buf = new StringBuilder()
+        .append(name);
+    appendArray(buf);
+    if (isMap()) {
+      buf.append(" {");
+      int i = 0;
+      for (ColumnMetadata child : members) {
+        if (i++ > 0) {
+          buf.append(", ");
+        }
+        if (child instanceof ProjectedColumn) {
+          buf.append(((ProjectedColumn) child).projectString());
+        } else {
+          buf.append(child.toString());
+        }
+      }
+      buf.append("}");
+    }
+    return buf.toString();
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/ProjectionSchemaTracker.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/ProjectionSchemaTracker.java
new file mode 100644
index 0000000..3e1373e
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/ProjectionSchemaTracker.java
@@ -0,0 +1,165 @@
+/*
+ * 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.v3.schema;
+
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.DynamicSchemaFilter.RowSchemaFilter;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.ScanProjectionParser.ProjectionParseResult;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.ScanSchemaResolver.SchemaType;
+import org.apache.drill.exec.physical.resultSet.impl.ProjectionFilter;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+
+/**
+ * Schema tracker for the "normal" case in which schema starts from a simple
+ * projection list of column names, optionally with a provided schema. The
+ * schema evolves by locating implicit columns, then having he reader define
+ * column types, and so on.
+ */
+public class ProjectionSchemaTracker extends AbstractSchemaTracker {
+
+  private final TupleMetadata projection;
+  private final boolean allowSchemaChange;
+  private int implicitInsertPoint;
+  private boolean allowMapAdditions = true;
+
+  public ProjectionSchemaTracker(TupleMetadata definedSchema,
+      ProjectionParseResult parseResult,
+      CustomErrorContext errorContext) {
+    super(errorContext);
+    this.projection = parseResult.dynamicSchema;
+    this.allowSchemaChange = false;
+    schema.copyFrom(definedSchema);
+    validateProjection(parseResult.dynamicSchema, definedSchema);
+    checkResolved();
+
+    ScanSchemaTracker.ProjectionType projType;
+    if (schema.size() == 0) {
+      projType = ScanSchemaTracker.ProjectionType.NONE;
+    } else {
+      projType = ScanSchemaTracker.ProjectionType.SOME;
+    }
+    schema.setProjectionType(projType);
+    this.implicitInsertPoint = -1;
+  }
+
+  public ProjectionSchemaTracker(ProjectionParseResult parseResult, boolean allowSchemaChange,
+      CustomErrorContext errorContext) {
+    super(errorContext);
+    this.projection = parseResult.dynamicSchema;
+    this.allowSchemaChange = allowSchemaChange;
+    this.schema.copyFrom(projection);
+
+    // Work out the projection type: wildcard, empty, or explicit.
+    ScanSchemaTracker.ProjectionType projType;
+    if (parseResult.isProjectAll()) {
+      projType = ScanSchemaTracker.ProjectionType.ALL;
+    } else if (projection.isEmpty()) {
+      projType = ScanSchemaTracker.ProjectionType.NONE;
+      this.isResolved = true;
+      this.allowMapAdditions = false;
+    } else {
+      projType = ScanSchemaTracker.ProjectionType.SOME;
+    }
+    this.schema.setProjectionType(projType);
+
+    // If wildcard, record the wildcard position.
+    this.schema.setInsertPoint(parseResult.wildcardPosn);
+    this.implicitInsertPoint = parseResult.wildcardPosn;
+  }
+
+  public void applyProvidedSchema(TupleMetadata providedSchema) {
+    boolean isStrict = SchemaUtils.isStrict(providedSchema);
+    new ScanSchemaResolver(schema,
+        isStrict ? SchemaType.STRICT_PROVIDED_SCHEMA : SchemaType.LENIENT_PROVIDED_SCHEMA,
+        true, errorContext)
+      .applySchema(providedSchema);
+    checkResolved();
+    if (isStrict) {
+      allowMapAdditions = false;
+    }
+  }
+
+  @Override
+  public void applyEarlyReaderSchema(TupleMetadata readerSchema) {
+    new ScanSchemaResolver(schema, SchemaType.EARLY_READER_SCHEMA, true, errorContext)
+      .applySchema(readerSchema);
+    checkResolved();
+  }
+
+  /**
+   * Set up a projection filter using the reader input schema returned
+   * from {@link #readerInputSchema()}.
+   * <ul>
+   * <li>If this is an empty projection (@{code SELECT COUNT(*)}), then
+   * noting can be projected at all.</li>
+   * <li>If this is an explicit projection (@code SELECT a, b)}, then
+   * the set of top-level columns is fixed, though the types are unknown.
+   * Maps allow new members depending on the map projection: a generic
+   * projection ({@code m}) allows new members, a specific projection
+   * ({@code m.a, m.b}) does not allow new members.</li>
+   * <li>If the schema has been resolved and is now fixed (closed), then
+   * no new columns are allowed either at the top level or in maps.</li>
+   * <li>If the schema is open ({@code SELECT *} for the first reader,
+   * or schema change is allowed in the second reader), and we have
+   * no columns, then just project everything.</li>
+   * <li>If the schema is open, but we have seen some columns, then
+   * columns can still be added, but existing columns must match the
+   * existing schema.</li>
+   * </ul>
+   * <p>
+   * Static filters handle the simple "none" and "starting from nothing
+   * all" cases. The dynamic schema filter handles the case of existing
+   * columns whether dynamic or static.
+   */
+  @Override
+  public ProjectionFilter projectionFilter(CustomErrorContext errorContext) {
+    switch (projectionType()) {
+      case ALL:
+        if (schema.size() == 0) {
+          return ProjectionFilter.PROJECT_ALL;
+        }
+        break;
+      case NONE:
+        return ProjectionFilter.PROJECT_NONE;
+      default:
+    }
+    return new RowSchemaFilter(schema, allowMapAdditions, errorContext);
+  }
+
+  @Override
+  public void applyReaderSchema(TupleMetadata readerOutputSchema,
+      CustomErrorContext errorContext) {
+    new ScanSchemaResolver(schema, SchemaType.READER_SCHEMA, allowMapAdditions, errorContext)
+        .applySchema(readerOutputSchema);
+    if (!allowSchemaChange) {
+      allowMapAdditions = false;
+      if (projectionType() == ProjectionType.ALL) {
+        schema.setProjectionType(ProjectionType.SOME);
+      }
+    }
+    checkResolved();
+  }
+
+  @Override
+  public void expandImplicitCol(ColumnMetadata resolved) {
+    Preconditions.checkArgument(SchemaUtils.isImplicit(resolved));
+    schema.insert(implicitInsertPoint++, resolved);
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/ScanProjectionParser.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/ScanProjectionParser.java
new file mode 100644
index 0000000..03033bf
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/ScanProjectionParser.java
@@ -0,0 +1,158 @@
+/*
+ * 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.v3.schema;
+
+import java.util.Collection;
+
+import org.apache.drill.common.expression.PathSegment;
+import org.apache.drill.common.expression.PathSegment.ArraySegment;
+import org.apache.drill.common.expression.PathSegment.NameSegment;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.Propertied;
+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.base.Preconditions;
+
+/**
+ * Parse the projection list into a dynamic tuple schema. Using
+ * an enhanced form of dynamic column which records projection list
+ * information (such as map members and array indexes.)
+ * <p>
+ * A wildcard project list can contain implicit columns in
+ * addition to the wildcard. The wildcard defines the
+ * <i>insert point</i>: the point at which reader-defined
+ * columns are inserted as found.
+ */
+public class ScanProjectionParser {
+
+  public static final String PROJECTION_TYPE_PROP = Propertied.DRILL_PROP_PREFIX + "proj-type";
+  public static final String PROJECT_ALL = "all";
+  public static final String PROJECT_NONE = "none";
+
+  public static class ProjectionParseResult {
+    public final int wildcardPosn;
+    public final TupleMetadata dynamicSchema;
+
+    public ProjectionParseResult(int wildcardPosn,
+        TupleMetadata dynamicSchema) {
+      this.wildcardPosn = wildcardPosn;
+      this.dynamicSchema = dynamicSchema;
+    }
+
+    public boolean isProjectAll() { return wildcardPosn != -1; }
+  }
+
+  private int wildcardPosn = -1;
+
+  public static ProjectionParseResult parse(Collection<SchemaPath> projList) {
+    if (projList == null) {
+      return SchemaUtils.projectAll();
+    }
+    if (projList.isEmpty()) {
+      return SchemaUtils.projectNone();
+    }
+    return new ScanProjectionParser().parseProjection(projList);
+  }
+
+  private ProjectionParseResult parseProjection(Collection<SchemaPath> projList) {
+    TupleMetadata tupleProj = new TupleSchema();
+    for (SchemaPath col : projList) {
+      parseMember(tupleProj, 0, col.getRootSegment());
+    }
+    return new ProjectionParseResult(wildcardPosn, tupleProj);
+  }
+
+  private void parseMember(TupleMetadata tuple, int depth, NameSegment nameSeg) {
+    String colName = nameSeg.getPath();
+    if (colName.equals(SchemaPath.DYNAMIC_STAR)) {
+      tuple.setProperty(PROJECTION_TYPE_PROP, PROJECT_ALL);
+      if (depth == 0) {
+        Preconditions.checkState(wildcardPosn == -1);
+        wildcardPosn = tuple.size();
+      }
+    } else {
+      ProjectedColumn col = project(tuple, nameSeg.getPath());
+      parseChildSeg(col, depth + 1, nameSeg);
+    }
+  }
+
+  protected ProjectedColumn project(TupleMetadata tuple, String colName) {
+    ColumnMetadata col = tuple.metadata(colName);
+    ProjectedColumn projCol;
+    if (col == null) {
+      projCol = new ProjectedColumn(colName);
+      tuple.addColumn(projCol);
+    } else {
+      projCol = (ProjectedColumn) col;
+      projCol.bumpRefCount();
+    }
+    return projCol;
+  }
+
+  private void parseChildSeg(ProjectedColumn column, int depth, PathSegment parentPath) {
+    if (parentPath.isLastPath()) {
+      parseLeaf(column, depth);
+    } else {
+      PathSegment seg = parentPath.getChild();
+      if (seg.isArray()) {
+        parseArraySeg(column, depth, (ArraySegment) seg);
+      } else {
+        parseMemberSeg(column, depth, (NameSegment) seg);
+      }
+    }
+  }
+
+  /**
+   * Parse a projection of the form {@code a}: that is, just a bare column.
+   */
+  private void parseLeaf(ProjectedColumn parent, int depth) {
+    if (parent.isSimple()) {
+      // Nothing to do
+    } else if (parent.isArray() && depth == 1) {
+      parent.projectAllElements();
+    } else if (parent.isMap()) {
+      parent.projectAllMembers();
+    }
+  }
+
+  private void parseArraySeg(ProjectedColumn column, int depth, ArraySegment arraySeg) {
+    boolean wasArray = column.isArray();
+    column.becomeArray(Math.max(depth, column.arrayDims()));
+
+    // Record only outermost dimension indexes
+    if (depth == 1) {
+      if (column.refCount() > 1 && !wasArray) {
+        column.projectAllElements();
+      } else {
+        column.addIndex(arraySeg.getIndex());
+      }
+    }
+    parseChildSeg(column, depth + 1, arraySeg);
+  }
+
+  private void parseMemberSeg(ProjectedColumn column, int depth, NameSegment memberSeg) {
+    if (column.refCount() > 1 && !column.isMap()) {
+      column.projectAllMembers();
+    }
+    TupleMetadata tuple = column.explicitMembers();
+    if (tuple != null) {
+      parseMember(tuple, depth, memberSeg);
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/ScanSchemaConfigBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/ScanSchemaConfigBuilder.java
new file mode 100644
index 0000000..24511a1
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/ScanSchemaConfigBuilder.java
@@ -0,0 +1,100 @@
+/*
+ * 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.v3.schema;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.EmptyErrorContext;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.ScanProjectionParser.ProjectionParseResult;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+
+/**
+ * Builds the configuration given to the {@link ScanSchemaTracker}.
+ */
+public class ScanSchemaConfigBuilder {
+  private TupleMetadata definedSchema;
+  private List<SchemaPath> projectionList;
+  private TupleMetadata providedSchema;
+  private boolean allowSchemaChange;
+  private CustomErrorContext errorContext;
+
+  public ScanSchemaConfigBuilder() {
+    projectionList = Collections.singletonList(SchemaPath.STAR_COLUMN);
+    allowSchemaChange = true;
+  }
+
+  public ScanSchemaConfigBuilder projection(List<SchemaPath> projectionList) {
+    this.projectionList = projectionList;
+    return this;
+  }
+
+  public ScanSchemaConfigBuilder definedSchema(TupleMetadata definedSchema) {
+    this.definedSchema = definedSchema;
+    return this;
+  }
+
+  public ScanSchemaConfigBuilder providedSchema(TupleMetadata providedSchema) {
+    this.providedSchema = providedSchema;
+    return this;
+  }
+
+  public ScanSchemaConfigBuilder allowSchemaChange(boolean flag) {
+    this.allowSchemaChange = flag;
+    return this;
+  }
+
+  public ScanSchemaConfigBuilder errorContext(CustomErrorContext errorContext) {
+    this.errorContext = errorContext;
+    return this;
+  }
+
+  public ScanSchemaTracker build() {
+    if (errorContext == null) {
+      errorContext = EmptyErrorContext.INSTANCE;
+    }
+    ProjectionParseResult result;
+    if (projectionList == null) {
+      result = null;
+    } else {
+      result = ScanProjectionParser.parse(projectionList);
+    }
+
+    if (providedSchema != null && SchemaUtils.isStrict(providedSchema)) {
+      allowSchemaChange = false;
+    }
+    if (definedSchema == null) {
+      ProjectionSchemaTracker tracker = new ProjectionSchemaTracker(result, allowSchemaChange, errorContext);
+      if (providedSchema != null) {
+        tracker.applyProvidedSchema(providedSchema);
+      }
+      return tracker;
+    } else {
+      if (!MetadataUtils.hasDynamicColumns(definedSchema)) {
+        SchemaBasedTracker tracker = new SchemaBasedTracker(definedSchema, errorContext);
+        tracker.validateProjection(result.dynamicSchema);
+        return tracker;
+      } else {
+        return new ProjectionSchemaTracker(definedSchema, result, errorContext);
+      }
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/ScanSchemaResolver.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/ScanSchemaResolver.java
new file mode 100644
index 0000000..6d1ec40
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/ScanSchemaResolver.java
@@ -0,0 +1,370 @@
+/*
+ * 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.v3.schema;
+
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.DynamicSchemaFilter.DynamicTupleFilter;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.MutableTupleSchema.ColumnHandle;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.ScanSchemaTracker.ProjectionType;
+import org.apache.drill.exec.physical.resultSet.impl.ProjectionFilter;
+import org.apache.drill.exec.physical.resultSet.impl.ProjectionFilter.ProjResult;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.DynamicColumn;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Resolves a schema against the existing scan schema.
+ * Expands columns by comparing the existing scan schema with
+ * a "revised" (provided or reader) schema, adjusting the scan schema
+ * accordingly. Maps are expanded recursively. Other columns
+ * must match types (concrete columns) or the type must match projection
+ * (for dynamic columns.)
+ * <ul>
+ * <li>Resolves a provided schema against the projection list.
+ * The provided schema can be strict (converts a wildcard into
+ * an explicit projection) or lenient (the reader can add
+ * additional columns to a wildcard.)</li>
+ * <li>Resolves an early reader schema against the projection list
+ * and optional provided schema.</li>
+ * <li>Resolves a reader output schema against a dynamic (projection
+ * list), concreted (provided or prior reader) schema) or combination.
+ * </ul>
+ * <p>
+ * In practice, the logic is simpler: given a schema (dynamic, concrete
+ * or combination), further resolve the schema using the input schema
+ * provided. Resolve dynamic columns, verify consistency of concrete
+ * columns.
+ * <p>
+ * Projected columns start as <i>dynamic</i> (no type). Columns
+ * are resolved to a known type as a schema identifies that type.
+ * Subsequent schemas are obligated to use that same type to avoid
+ * an inconsistent schema change downstream.
+ * <p>
+ * Expands columns by comparing the existing scan schema with
+ * a "revised" (provided or reader) schema, adjusting the scan schema
+ * accordingly. Maps are expanded recursively. Other columns
+ * must match types (concrete columns) or the type must match projection
+ * (for dynamic columns.)
+ * <p>
+ * A "resolved" projection list is a list of concrete columns: table
+ * columns, nulls, file metadata or partition metadata. An unresolved list
+ * has either table column names, but no match, or a wildcard column.
+ * <p>
+ * The idea is that the projection list moves through stages of resolution
+ * depending on which information is available. An "early schema" table
+ * provides schema information up front, and so allows fully resolving
+ * the projection list on table open. A "late schema" table allows only a
+ * partially resolved projection list, with the remainder of resolution
+ * happening on the first (or perhaps every) batch.
+ */
+public class ScanSchemaResolver {
+  private static final Logger logger = LoggerFactory.getLogger(ScanSchemaResolver.class);
+
+  /**
+   * Indicates the source of the schema to be analyzed.
+   * Each schema type has subtly different rules. The
+   * schema type allows us to inject those differences inline
+   * within the resolution process. Also, each schema caries
+   * a tag used for error reporting.
+   */
+  public enum SchemaType {
+    STRICT_PROVIDED_SCHEMA("Provided"),
+    LENIENT_PROVIDED_SCHEMA("Provided"),
+    EARLY_READER_SCHEMA("Reader"),
+    READER_SCHEMA("Reader"),
+    MISSING_COLS("Missing columns");
+
+    private final String source;
+
+    SchemaType(String source) {
+      this.source = source;
+    }
+
+    public String source() {
+      return source;
+    }
+  }
+
+  private final MutableTupleSchema schema;
+  private final SchemaType mode;
+  private final boolean isProjectAll;
+  private final boolean allowMapAdditions;
+  private final String source;
+  private final CustomErrorContext errorContext;
+
+  public ScanSchemaResolver(MutableTupleSchema schema, SchemaType mode,
+      boolean allowMapAdditions,
+      CustomErrorContext errorContext) {
+    this.schema = schema;
+    this.isProjectAll = schema.projectionType() == ProjectionType.ALL;
+    this.mode = mode;
+    this.errorContext = errorContext;
+    this.allowMapAdditions = allowMapAdditions;
+    this.source = mode.source();
+  }
+
+  public void applySchema(TupleMetadata sourceSchema) {
+    switch (schema.projectionType()) {
+      case ALL:
+      case SOME:
+        projectSchema(sourceSchema);
+        break;
+      default:
+        // Do nothing
+    }
+    if (mode == SchemaType.STRICT_PROVIDED_SCHEMA && isProjectAll) {
+      schema.setProjectionType(ScanSchemaTracker.ProjectionType.SOME);
+    }
+  }
+
+  /**
+   * A project list can contain implicit columns in
+   * addition to the wildcard. The wildcard defines the
+   * <i>insert point</i>: the point at which reader-defined
+   * columns are inserted as found.
+   */
+  private void projectSchema(TupleMetadata sourceSchema) {
+    for (ColumnMetadata colSchema : sourceSchema) {
+      ColumnHandle existing = schema.find(colSchema.name());
+       if (existing == null) {
+        insertColumn(colSchema);
+      } else {
+        mergeColumn(existing, colSchema);
+      }
+    }
+  }
+
+  /**
+   * Insert a new column into the schema at the wildcard insertion point.
+   * Columns can generally only be inserted for a wildcard, and only when
+   * schema change is allowed. A number of special cases occur for the
+   * various kinds of schemas.
+   */
+  private void insertColumn(ColumnMetadata col) {
+    switch (mode) {
+      case READER_SCHEMA:
+        if (!isProjectAll) {
+          throw new IllegalStateException(
+              "Reader should not have projected an unprojected column: " + col.name());
+        }
+        break;
+      case EARLY_READER_SCHEMA:
+      case LENIENT_PROVIDED_SCHEMA:
+      case STRICT_PROVIDED_SCHEMA:
+        if (!isProjectAll || SchemaUtils.isExcludedFromWildcard(col)) {
+          return;
+        }
+        break;
+      case MISSING_COLS:
+        throw new IllegalStateException("Missing columns should not add new columns");
+      default:
+        throw new IllegalStateException(mode.name());
+    }
+    ColumnMetadata copy = col.copy();
+    schema.insert(copy);
+
+    // This is a top-level column inserted from a wildcard. If the column
+    // is a map, then project all its members. But, don't do that for
+    // a strict schema. ("Strict" means only allow members in the provided
+    // schema.)
+    if (copy.isMap() && mode != SchemaType.STRICT_PROVIDED_SCHEMA) {
+      SchemaUtils.markProjectAll(copy);
+    }
+  }
+
+  /**
+   * Merge an incoming column with an existing column which can either be
+   * dynamic or concrete. Special cases occur for implicit columns which are
+   * independent of reader schema, but which reside in the same namespace,
+   * causing potential conflicts.
+   */
+  private void mergeColumn(ColumnHandle col, ColumnMetadata colSchema) {
+    switch (mode) {
+      case LENIENT_PROVIDED_SCHEMA:
+      case STRICT_PROVIDED_SCHEMA:
+        // With a wilcard, there should be no existing column unless
+        // the planner projected an implicit column and the provided
+        // schema defines that same implicit column.
+        if (isProjectAll && !SchemaUtils.isImplicit(colSchema)) {
+          throw UserException.validationError()
+            .message("Provided schema column name conflicts with presumed implicit column name")
+            .addContext("Column", colSchema.name())
+            .addContext(errorContext)
+            .build(logger);
+        }
+        break;
+      case EARLY_READER_SCHEMA:
+        // If the reader offers a column which duplicates an implicit column,
+        // act as if the column is not projected, but give a warning since
+        // the user might expect the column to be projected in a wildcard.
+        if (col.isImplicit()) {
+           logger.warn("Column {} shadows an implicit column of the same name: ignored",
+              colSchema.name());
+           return;
+        }
+        break;
+      default:
+        // The reader should not project a column with the same name as an
+        // implicit column. The projection filter should have prevented it.
+        // If projection does occur, we cannot accept the column and have
+        // no way to dispose of the unwanted vector.
+        if (col.isImplicit()) {
+          throw UserException.validationError()
+            .message("Reader column conflicts an implicit column, should not have been projected")
+            .addContext("Column", colSchema.name())
+            .addContext(errorContext)
+            .build(logger);
+        }
+    }
+    if (col.column().isDynamic()) {
+      mergeColumnWithDynamic(col, colSchema);
+    } else {
+      mergeWithConcrete(col.column(), colSchema);
+    }
+  }
+
+  /**
+   * Merge a resolved column with a dynamic column (from the project list or a dynamic
+   * defined schema). Verify consistency with the projection. Should have already
+   * been done by the projection filter for reader output columns, done here for all
+   * other schema types.
+   */
+  private void mergeColumnWithDynamic(ColumnHandle existing, ColumnMetadata revised) {
+    DynamicColumn existingCol = (DynamicColumn) existing.column();
+    if (existingCol instanceof ProjectedColumn) {
+      SchemaUtils.verifyCompatibility((ProjectedColumn) existingCol, revised,
+          source, errorContext);
+    }
+    if (existingCol.isMap() || revised.isMap()) {
+      schema.replace(existing, createMap(existingCol, revised));
+    } else {
+      schema.resolve(existing, revised.copy());
+    }
+  }
+
+  /**
+   * Merge an incoming column with an existing resolved column. Non-map columns
+   * must match. Maps are merged recursively.
+   */
+  private void mergeWithConcrete(ColumnMetadata existing,
+      ColumnMetadata revised) {
+    SchemaUtils.verifyConsistency(existing, revised, source, errorContext);
+    if (existing.isMap()) {
+      ProjectionFilter filter = new DynamicTupleFilter(existing.tupleSchema(),
+          allowMapAdditions, errorContext, source);
+      expandMapProjection(existing.tupleSchema(), filter, revised.tupleSchema());
+    }
+  }
+
+  /**
+   * Create a map column. The map might have an explicit projection
+   * ({@code m.a, m.b}). To ensure consistency with reader behavior, use the same
+   * projection filter as the reader to determine which provided or early reader schema
+   * columns to project.
+   */
+  private ColumnMetadata createMap(DynamicColumn projection,
+      ColumnMetadata revised) {
+    return createMap(projection,
+        DynamicTupleFilter.filterFor(projection, allowMapAdditions, errorContext, source),
+        revised);
+  }
+
+  /**
+   * Recursively create a map, including nested maps.
+   */
+  private ColumnMetadata createMap(DynamicColumn projection, ProjectionFilter filter,
+      ColumnMetadata revised) {
+    ColumnMetadata map = revised.cloneEmpty();
+    SchemaUtils.mergeColProperties(map, projection);
+    SchemaUtils.mergeColProperties(map, revised);
+    copyDynamicMembers(map, projection);
+
+    // When resolving a generic column to a map, the map is treated
+    // as "map.*". That is, we can add additional columns later
+    // (assuming the scan allows it.) However, if this is a strict schema,
+    // then strict means no additional columns are allowed.
+    if (!projection.isMap() && mode != SchemaType.STRICT_PROVIDED_SCHEMA) {
+      SchemaUtils.markProjectAll(map);
+    }
+    expandMapProjection(map.tupleSchema(), filter, revised.tupleSchema());
+    return map;
+  }
+
+  private void copyDynamicMembers(ColumnMetadata map, DynamicColumn projection) {
+    if (projection.isMap()) {
+      TupleMetadata mapSchema = map.tupleSchema();
+      for (ColumnMetadata col : projection.tupleSchema()) {
+        mapSchema.addColumn(col.copy());
+      }
+    }
+  }
+
+ /**
+   * Given an existing map, a projection filter, and an actual
+   * reader output, update the existing map with the reader schema,
+   * gated on the projection schema. Note that the projection schema
+   * may not be needed it the reader schema followed the projection
+   * filter which was based on the projection map.
+   */
+  private void expandMapProjection(TupleMetadata scanSchema,
+      ProjectionFilter filter,
+      TupleMetadata revisedSchema) {
+    for (ColumnMetadata readerCol : revisedSchema) {
+      resolveMember(scanSchema, filter.projection(readerCol), readerCol);
+    }
+  }
+
+  private void resolveMember(TupleMetadata scanSchema, ProjResult result,
+      ColumnMetadata readerCol) {
+    ColumnMetadata schemaCol = result.projection;
+    if (!result.isProjected) {
+      switch (mode) {
+      case EARLY_READER_SCHEMA:
+      case LENIENT_PROVIDED_SCHEMA:
+      case STRICT_PROVIDED_SCHEMA:
+        break;
+      case READER_SCHEMA:
+        if (!allowMapAdditions) {
+          throw new IllegalStateException("Reader should not have projected column: " + readerCol.name());
+        }
+        break;
+      default:
+        throw new IllegalStateException(mode.name());
+      }
+    } else if (schemaCol == null) {
+      ColumnMetadata copy = readerCol.copy();
+      if (readerCol.isMap()) {
+        SchemaUtils.markProjectAll(copy);
+      }
+      scanSchema.addColumn(copy);
+    } else if (schemaCol.isDynamic()) {
+      if (schemaCol.isMap()) {
+        scanSchema.replace(createMap((ProjectedColumn) schemaCol,
+            result.mapFilter, readerCol));
+      } else {
+        scanSchema.replace(readerCol.copy());
+      }
+    } else if (schemaCol.isMap()) {
+      expandMapProjection(schemaCol.tupleSchema(),
+          result.mapFilter, readerCol.tupleSchema());
+    } // else cols are identical simple
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/ScanSchemaTracker.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/ScanSchemaTracker.java
new file mode 100644
index 0000000..7d1aa30
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/ScanSchemaTracker.java
@@ -0,0 +1,511 @@
+/*
+ * 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.v3.schema;
+
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.impl.ProjectionFilter;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Computes <i>scan output schema</i> from a variety of sources.
+ * <p>
+ * The scan operator output schema can be <i>defined</i> or <i>dynamic.</i>
+ *
+ * <h4>Defined Schema</h4>
+ *
+ * The planner computes a defined schema from metadata, as in a typical
+ * query engine. A defined schema defines the output schema directly:
+ * the defined schema <b>is</b> the output schema. Drill's planner does not
+ * yet support a defined schema, but work is in progress to get there for
+ * some cases.
+ * <p>
+ * With a defined schema, the reader is given a fully-defined schema and
+ * its job is to produce vectors that match the given schema. (The details
+ * are handled by the {@link ResultSetLoader}.)
+ * <p>
+ * At present, since the planner does not actually provide a defined schema,
+ * we support it in this class, and verify that the defined schema, if provided,
+ * exactly matches the names in the project list in the same order.
+ *
+ * <h4>Dynamic Schema</h4>
+ *
+ * A dynamic schema is one defined at run time: the traditional Drill approach.
+ * A dynamic schema starts with a <i>projection list</i> : a list of column names
+ * without types.
+ * This class converts the project list into a dynamic reader schema which is
+ * a schema in which each column has the type {@code LATE}, which basically means
+ * "a type to be named later" by the reader.
+ *
+ * <h4>Hybrid Schema</h4>
+ *
+ * Some readers support a <i>provided schema</i>, which is an concept similar to,
+ * but distinct from, a defined schema. The provided schema provides <i>hints</i>
+ * about a schema. At present, it
+ * is an extra; not used or understood by the planner. Thus, the projection
+ * list is independent of the provided schema: the lists may be disjoint.
+ * <p>
+ * With a provided schema, the project list defines the output schema. If the
+ * provided schema provides projected columns, then the provided schema for those
+ * columns flow to the output schema, just as for a defined schema. Similarly, the
+ * reader is given a defined schema for those columns.
+ * <p>
+ * Where a provided schema differs is that the project list can include columns
+ * not in the provided schema, such columns act like the dynamic case: the reader
+ * defines the column type.
+ *
+ * <h4>Projection Types</h4>
+ *
+ * Drill will pass in a project list which is one of three kinds:
+ * <p><ul>
+ * <li>{@code >SELECT *}: 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>
+ * <li>{@code >SELECT a, b, c, ...}: Project a specific set of columns, identified by
+ * case-insensitive name. The output row uses the names from the SELECT list,
+ * but types from the data source. Columns appear in the row in the order
+ * specified by the {@code SELECT}.</li>
+ * <li{@code >SELECT ...}: Project nothing, occurs in {@code >SELECT COUNT(*)}
+ * type queries. The provided projection list contains no (table) columns, though
+ * it may contain metadata columns.</li>
+ * </ul>
+ * Names in the project list can reference any of five distinct types of output
+ * columns:
+ * <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
+ * projection planner will determine if the column exists, or must be filled
+ * in with a null column.</li>
+ * <li>The generic data source columns array: {@code >columns}, or optionally
+ * specific members of the {@code >columns} array such as {@code >columns[1]}.
+ * (Supported only by specific readers.)</li>
+ * <li>Implicit columns: {@code >fqn}, {@code >filename}, {@code >filepath}
+ * and {@code >suffix}. These reference
+ * parts of the name of the file being scanned.</li>
+ * <li>Partition columns: {@code >dir0}, {@code >dir1}, ...: These reference
+ * parts of the path name of the file.</li>
+ * </ul>
+ *
+ * <h4>Empty Schema</h4>
+ *
+ * A special case occurs if the projection list is empty which indicates that
+ * the query is a {@code COUNT(*)}: we need only a count of columns, but none
+ * of the values. Implementation of the count is left to the specific reader
+ * as some can optimize this case. The output schema may include a single
+ * dummy column. In this case, the first batch defines the schema expected
+ * from all subsequent readers and batches.
+ *
+ * <h4>Implicit Columns</h4>
+ *
+ * The project list can contain implicit columns for data sources which support
+ * them. Implicit columns are disjoint from data source columns and are provided
+ * by Drill itself. This class effectively splits the projection list into
+ * a set of implicit columns, and the remainder of the list which are the
+ * reader columns.
+ *
+ * <h4>Reader Input Schema</h4>
+ *
+ * The various forms of schema above produce a <i>reader input schema</i>:
+ * the schema given to the reader. The reader input schema is the set of
+ * projected columns, minus implicit columns, along with available type
+ * information.
+ * <p>
+ * If the reader can produce only one type
+ * for each column, then the provided or defined schema should already specify
+ * that type, and the reader can simply ignore the reader input schema. (This
+ * feature allows this scheme to be compatible with older readers.)
+ * <p>
+ * However, if the reader can convert a column to multiple types, then the
+ * reader should use the reader input schema to choose a type. If the input
+ * schema is dynamic (type is {@code LATE}), then the reader chooses the
+ * column type and should chose the "most natural" type.
+ *
+ * <h4>Reader Output Schema</h4>
+ *
+ * The reader proceeds to read a batch of data, choosing types for dynamic
+ * columns. The reader may provide a subset of projected columns if, say
+ * the reader reads an older file that is missing some columns or (for a
+ * dynamic schema), the user specified columns which don't actually exist.
+ * <p>
+ * The result is the <i>reader output schema</i>: a subset of the reader
+ * input schema in which each included column has a concrete type. (The
+ * reader may have provided extra columns. In this case, the
+ * {@code ResultSetLoader} will have ignored those columns, providing a
+ * dummy column writer, and omitting non-projected columns from the reader
+ * output schema.)
+ * <p>
+ * The reader output schema is provided to this class which resolves any
+ * dynamic columns to the concrete type provided by the reader. If the
+ * column was already resolved, this class ensures that the reader's
+ * column type matches the resolved type to prevent column type changes.
+ *
+ * <h4>Dynamic Wildcard Schema</h4>
+ *
+ * Traditional query planners resolve the wildcard ({@code *}) in the
+ * planner. When using a dynamic schema, Drill resolves the wildcard at
+ * run time. In this case, the reader input schema is empty and the reader
+ * defines the entire set of columns: names and types. This class then
+ * replaces the wildcard with the columns from the reader.
+ *
+ * <h4>Missing Columns</h4>
+ *
+ * When the reader output schema is a subset of the reader input schema,
+ * the we have a set of <i>missing columns</i> (also called "null columns").
+ * A part of the scan framework must invent vectors for these columns. If
+ * the type is available, then that is the type used, otherwise the missing
+ * column handler must invent a type (such as the classic
+ * {@code nullable INT} historically used.) If the mode is
+ * nullable, the column is filled with nulls. If non-nullable, the column
+ * is filled with a default value. All of this work happens outside of
+ * this class.
+ * <p>
+ * The missing column handler defined its own output schema which is
+ * resolved by this class identical to how the reader schema is resolved.
+ * The result is that all columns are now resolved to a concrete type.
+ * <p>
+ * Missing columns may be needed even for a wildcard if a first reader
+ * discovered 3 columns, say, but a later reader encounters only two of
+ * them.
+ *
+ * <h4>Subsequent Readers and Schema Changes</h4>
+ *
+ * All of the above occurs during the first batch of data. After that,
+ * the schema is fully defined: subsequent readers will encounter only
+ * a fully defined schema, which it must handle the same as if the scan
+ * was given a defined schema.
+ * <p>
+ * This rule works file for an explicit project list. However, if the
+ * project list is dynamic, and contains a wildcard, then the reader
+ * defines the output schema. What happens if a reader adds columns
+ * (or a second or later reader discovers new columns)? Traditionally,
+ * Drill simply adds those columns and sends a {@code OK_NEW_SCHEMA}
+ * (schema change) downstream for other operators to deal with.
+ * <p>
+ * This class supports the traditional approach as an option. This class
+ * also supports a more rational, strict rule: the schema is fixed after
+ * the first batch. That is, the first batch defines a <i>schema commit
+ * point</i> after which the scan agrees not to change the schema. In
+ * this scenario, the first batch defines a schema (and project list)
+ * given to all subsequent readers. Any new columns are ignored (with
+ * a warning in the log.)
+ *
+ * <h4>Output Schema</h4>
+ *
+ * All of the above contribute to the <i>output schema</i>: the schema
+ * sent downstream to the next operator. All of the above work is done to
+ * either:
+ * <ul>
+ * <li>Pass the defined schema to the output, with the reader (and missing
+ * columns handler) producing columns that match that schema.</li>
+ * <li>Expand the dynamic schema with details provided by the reader
+ * (and missing columns hander), including the actual set of columns if
+ * the dynamic schema includes a wildcard.</li>
+ * </ul>
+ * <p>
+ * Either way, the result is a schema which describes the actual vectors
+ * sent downstream.
+ *
+ * <h4>Consumers</h4>
+ *
+ * Information from this class is used in multiple ways:
+ * <ul>
+ * <li>A project list is given to the {@code ResultSetLoader} to specify which
+ * columns to project to vectors, and which to satisfy with a dummy column
+ * reader.</li>
+ * <li>The reader, via the {code SchemaNegotiator} uses the reader input
+ * schema.</li>
+ * <li>The reader, via the {@code ResultSetLoader} provides the reader output
+ * schema.</li>
+ * <li>An implicit column manager handles the various implicit and partition
+ * directory columns: identifying them then later providing vector values.</li>
+ * <li>A missing columns handler fills in missing columns.</li>
+ * </ul>
+ *
+ * <h4>Design</h4>
+ *
+ * Schema resolution is a set of layers of choices. Each level and choice is
+ * represented by a class: virtual method pick the right path based on class
+ * type rather than using a large collection of if-statements.
+ *
+ * <h4>Maps</h4>
+ *
+ * Maps present a difficult challenge. Drill allows projection within maps
+ * and we wish to exploit that in the scan. For example: {@code m.a}. The
+ * column state classes provide a map class. However, the projection notation
+ * is ambiguous: {@code m.a} could be a map {@code `m`} with a child column
+ * {@code 'a'}. Or, it could be a {@code DICT} with a {code VARCHAR} key.
+ * <p>
+ * To handle this, if we only have the project list, we use an unresolved
+ * column state, even if the projection itself has internal structure. We
+ * use a projection-based filter in the {@code ResultSetLoader} to handle
+ * the ambiguity. The projection filter, when presented with the reader's
+ * choice for column type, will check if that type is consistent with projection.
+ * If so, the reader will later present the reader output schema which we
+ * use to resolve the projection-only unresolved column to a map column.
+ * (Or, if the column turns out to be a {@code DICT}, to a simple unresolved
+ * column.)
+ * <p>
+ * If the scan contains a second reader, then the second reader is given a
+ * stricter form of projection filter: one based on the actual {@code MAP}
+ * (or {@code DICT}) column.
+ * <p>
+ * If a defined or provided schema is available, then the schema tracker
+ * does have sufficient information to resolve the column directly to a
+ * map column, and the first reader will have the strict projection filter.
+ * <p>
+ * A user can project a map column which does not actually exist (or, at
+ * least, is not known to the first reader.) In that case, the missing
+ * column logic applies, but within the map. As a result, a second reader
+ * may encounter a type conflict if it discovers the previously-missing
+ * column, and finds that the default type conflicts with the real type.
+ * <p>
+ * @see {@link ImplicitColumnExplorer}, the class from which this class
+ * evolved
+ */
+public interface ScanSchemaTracker {
+
+  enum ProjectionType {
+
+    /**
+     * This a wildcard projection. The project list may include
+     * implicit columns in addition to the wildcard.
+     */
+    ALL,
+
+    /**
+     * This is an empty projection, such as for a COUNT(*) query.
+     * No implicit columns will appear in such a scan.
+     */
+    NONE,
+
+    /**
+     * Explicit projection with a defined set of columns.
+     */
+    SOME
+  }
+
+  ProjectionType projectionType();
+
+  /**
+   * Is the scan schema resolved? The schema is resolved depending on the
+   * complex lifecycle explained in the class comment. Resolution occurs
+   * when the wildcard (if any) is expanded, and all explicit projection
+   * columns obtain a definite type. If schema change is disabled, the
+   * schema will not change once it is resolved. If schema change is allowed,
+   * then batches or readers may extend the schema, triggering a schema
+   * change, and so the scan schema may move from one resolved state to
+   * another.
+   * <p>
+   * The schema will be fully resolved after the first batch of data arrives
+   * from a reader (since the reader lifecycle will then fill in any missing
+   * columns.) The schema may be resolved sooner (such as if a strict provided
+   * schema, or an early reader schema is available and there are no missing
+   * columns.)
+   *
+   * @return {@code} if the schema is resolved, and hence the
+   * {@link #outputSchema()} is available, {@code false} if the schema
+   * contains one or more dynamic columns which are not yet resolved.
+   */
+  boolean isResolved();
+
+  /**
+   * Gives the output schema version which will start at some arbitrary
+   * positive number.
+   * <p>
+   * If schema change is allowed, the schema version allows detecting
+   * schema changes as the scan schema moves from one resolved state to
+   * the next. Each schema will have a unique, increasing version number.
+   * A schema change has occurred if the version is newer than the previous
+   * output schema version.
+   *
+   * @return the schema version. The absolute number is not important,
+   * rather an increase indicates one or more columns were added at the
+   * top level or within a map at some nesting level
+   */
+  int schemaVersion();
+
+  /**
+   * Drill defines a wildcard to expand not just reader columns, but also
+   * partition columns. When the implicit column handlers sees that the
+   * query has a wildcard (by calling {@link #isProjectAll()}), the handler
+   * then determines which partition columns are needed and calls this
+   * method to add each one.
+   */
+  void expandImplicitCol(ColumnMetadata resolved);
+
+  /**
+   * Indicate that implicit column parsing is complete. Returns the implicit
+   * columns as identified by the implicit column handler, in the order of the
+   * projection list. Implicit columns do not appear in a reader input schema,
+   * and it is an error for the reader to produce such columns.
+   *
+   * @return a sub-schema of only implicit columns, in the order in which
+   * they appear in the output schema
+   */
+  TupleMetadata applyImplicitCols();
+
+  /**
+   * If a reader can define a schema before reading data, apply that
+   * schema to the scan schema. Allows the scan to report its output
+   * schema before the first batch of data if the scan schema becomes
+   * resolved after the early reader schema.
+   */
+  void applyEarlyReaderSchema(TupleMetadata readerSchema);
+
+  /**
+   * The schema which the reader should produce. Depending on the type of
+   * the scan (specifically, if {@link #isProjectAll()} is {@code true}),
+   * the reader may produce additional columns beyond those in the the
+   * reader input schema. However, for any batch, the reader, plus the
+   * missing columns handler, must produce all columns in the reader input
+   * schema.
+   * <p>
+   * Formally:<pre><code>
+   * reader input schema = output schema - implicit col schema
+   * </code></pre>
+   *
+   * @return the sub-schema which includes those columns which the reader
+   * should provide, excluding implicit columns
+   */
+  TupleMetadata readerInputSchema();
+
+  /**
+   * Identifies the missing columns given a reader output schema. The reader
+   * output schema are those columns which the reader actually produced.
+   * <p>
+   * Formally:<pre><code>
+   * missing cols = reader input schema - reader output schema
+   * </code></pre>
+   * <p>
+   * The reader output schema can contain extra, newly discovered columns.
+   * Those are ignored when computing missing columns. Thus, the subtraction
+   * is set subtraction: remove columns common to the two sets.
+   *
+   * @code the sub-schema of the reader schema which excludes the columns
+   * which the reader provided. The result are the "missing" columns which
+   * have no values in a given batch of rows
+   */
+  TupleMetadata missingColumns(TupleMetadata readerOutputSchema);
+
+  /**
+   * Returns the scan output schema which is a somewhat complicated
+   * computation that depends on the projection type.
+   * <p>
+   * For a wildcard schema:<pre><code>
+   * output schema = implicit cols U reader output schema
+   * </code></pre>
+   * <p>
+   * For an explicit projection:<pre><code>
+   * output schema = projection list
+   * </code></pre>
+   * Where the projection list is augmented by types from the
+   * provided schema, implicit columns or readers.
+   * <p>
+   * A defined schema <i>is</i> the output schema, so:<code><pre>
+   * output schema = defined schema
+   * </pre></code>
+   *
+   * @return the complete output schema provided by the scan to downstream
+   * operators. Includes both reader and implicit columns, in the order
+   * of the projection list or, for a wildcard, in the order of the first
+   * reader
+   */
+  TupleMetadata outputSchema();
+
+  /**
+   * A reader is responsible for reading columns in the reader input schema.
+   * A reader may read additional columns. The projection filter is passed to
+   * the {@link ResultSetLoader} to determine which columns should be projected,
+   * allowing the reader to be blissfully ignorant of which columns are needed.
+   * The result set loader provides a dummy reader for unprojected columns.
+   * (A reader can, via the result set loader, find if a column is projected if
+   * doing so helps reader efficiency.)
+   * <p>
+   * The projection filter is the first line of defense for schema conflicts.
+   * The {code ResultSetLoader} will query the filter with a full column
+   * schema. If that schema conflicts with the scan schema for that column,
+   * this method will raise a {@code UserException}, which typically indicates
+   * a programming error, or a very odd data source in which a column changes
+   * types between batches.
+   *
+   * @param errorContext the reader-specific error context to use if
+   * errors are found
+   * @return a filter used to decide which reader columns to project during
+   * reading
+   */
+  ProjectionFilter projectionFilter(CustomErrorContext errorContext);
+
+  /**
+   * Once a reader has read a batch, the reader will have provided a type
+   * for each projected column which the reader knows about. For a wildcard
+   * projection, the reader will have added all the columns that it found.
+   * This call takes the reader output schema and merges it with the current
+   * scan schema to resolve dynamic types to concrete types and to add
+   * newly discovered columns.
+   * <p>
+   * The process can raise an exception if the reader projects a column that
+   * it shouldn't (which is not actually possible because of the way the
+   * {@code ResultSetLoader} works.) An error can also occur if the reader
+   * provides a type different than that already defined in the scan schema
+   * by a defined schema, a provided schema, or a previous reader in the same
+   * scan. In such cases, the reader is expected to have converted its input
+   * type to the specified type, which was presumably selected because the
+   * reader is capable of the required conversion.
+   *
+   * @param readerOutputSchema the actual schema produced by a reader when
+   * reading a record batch
+   * @param errorContext the reader-specific error context to use if
+   * errors are found
+   */
+  void applyReaderSchema(TupleMetadata readerOutputSchema,
+      CustomErrorContext errorContext);
+
+  /**
+   * The missing column handler obtains the list of missing columns from
+   * {@link #missingColumns()}. Depending on the scan lifecycle, some of the
+   * columns may have a type, others may be dynamic. The missing column handler
+   * chooses a type for any dynamic columns, then calls this method to tell
+   * the scan schema tracker the now-resolved column type.
+   * <p>
+   * Note: a goal of the provided/defined schema system is to avoid the need
+   * to guess types for missing columns since doing so quite often leads
+   * to problems further downstream in the query. Ideally, the type of missing
+   * columns will be known (via the provided or defined schema) to avoid
+   * such conflicts.
+   */
+  void resolveMissingCols(TupleMetadata missingCols);
+
+  /**
+   * The scan-level error context used for errors which may occur before the
+   * first reader starts. The reader will provide a more detailed error context
+   * that describes what is being read.
+   *
+   * @return the scan-level error context
+   */
+  CustomErrorContext errorContext();
+
+  /**
+   * Returns the internal scan schema. Primarily for testing.
+   * @return the internal mutable scan schema
+   */
+  @VisibleForTesting
+  MutableTupleSchema internalSchema();
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/SchemaBasedTracker.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/SchemaBasedTracker.java
new file mode 100644
index 0000000..24beb38
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/SchemaBasedTracker.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.scan.v3.schema;
+
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.DynamicSchemaFilter.RowSchemaFilter;
+import org.apache.drill.exec.physical.resultSet.impl.ProjectionFilter;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+
+/**
+ * Simple "tracker" based on a defined, fixed schema. The only resolution
+ * needed is to identify which columns, if any, are implicit. Readers are
+ * ignorant of this state: readers may still produce a subset of the defined
+ * columns and so missing columns may be needed. However, readers cannot
+ * add to the set of output columns, nor change their types.
+ */
+public class SchemaBasedTracker extends AbstractSchemaTracker {
+
+  private final TupleMetadata definedSchema;
+
+  public SchemaBasedTracker(TupleMetadata definedSchema, CustomErrorContext errorContext) {
+    super(errorContext);
+    this.definedSchema = definedSchema;
+    schema.copyFrom(definedSchema);
+    checkResolved();
+
+    // If not resolved, should not have used this tracker.
+    Preconditions.checkState(isResolved);
+
+    ScanSchemaTracker.ProjectionType projType;
+    if (schema.size() == 0) {
+      projType = ScanSchemaTracker.ProjectionType.NONE;
+    } else {
+      projType = ScanSchemaTracker.ProjectionType.SOME;
+    }
+    schema.setProjectionType(projType);
+  }
+
+  public void validateProjection(TupleMetadata projection) {
+    if (projection == null) {
+      return;
+    }
+    validateProjection(projection, definedSchema);
+  }
+
+  @Override
+  public void applyEarlyReaderSchema(TupleMetadata readerSchema) { }
+
+  /**
+   * Set up a projection filter using the defined schema
+   */
+  @Override
+  public ProjectionFilter projectionFilter(CustomErrorContext errorContext) {
+    switch (projectionType()) {
+      case NONE:
+        return ProjectionFilter.PROJECT_NONE;
+      case SOME:
+        return new RowSchemaFilter(schema, false, errorContext);
+      default:
+        throw new IllegalStateException(projectionType().name());
+    }
+  }
+
+  @Override
+  public void applyReaderSchema(TupleMetadata readerOutputSchema,
+      CustomErrorContext errorContext) {
+    // TODO: Validate reader output is a subset of the schema
+  }
+
+  @Override
+  public void expandImplicitCol(ColumnMetadata resolved) {
+    throw new IllegalStateException("Can't expand a defined schema.");
+  }
+
+  @Override
+  public int schemaVersion() { return 1; }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/SchemaUtils.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/SchemaUtils.java
new file mode 100644
index 0000000..fa8d8fd
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/SchemaUtils.java
@@ -0,0 +1,237 @@
+/*
+ * 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.v3.schema;
+
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.resultSet.project.RequestedColumn;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.Propertied;
+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.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Set of schema utilities that don't fit well as methods on the column
+ * or tuple classes.
+ * <p>
+ * Provides methods to check if a column is consistent with the projection
+ * requested for a query. Used for scans: the reader offers certain columns
+ * and the scan operator must decide whether to accept them, and if so,
+ * if the column that has actually appeared is consistent with the projection
+ * schema path provided by the planner. An obvious example is if projection
+ * asks for {@code a[0]} (and array), but the reader offer up {@code a}
+ * as a non-array column.
+ * <p>
+ * Checks are reasonable, but not complete. Particularly in the {@code DICT}
+ * case, projection depends on multiple factors, such as the type of the
+ * key and values. This class does not (yet) handle that complexity.
+ * Instead, the goal is no false negatives for the complex cases, while
+ * catching the simple cases.
+ * <p>
+ * The Project operator or other consuming operator is the final arbitrator
+ * of whether a particular column satisfies a particular projection. This
+ * class tries to catch those errors early to provide better error
+ * messages.
+ */
+public class SchemaUtils {
+  protected static final Logger logger = LoggerFactory.getLogger(SchemaUtils.class);
+
+  /**
+   * Check if the given read column is consistent with the projection requested for
+   * that column. Does not handle subtleties such as DICT key types, actual types
+   * in a UNION, etc.
+   *
+   * @param colReq the column-level projection description
+   * @param readCol metadata for the column which the reader has actually
+   * produced
+   * @return {@code true} if the column is consistent with projection (or if the
+   * column is too complex to check), {@code false} if the column is not
+   * consistent and represents an error case. Also returns {@code true} if
+   * the column is not projected, as any type of column can be ignored
+   */
+  public static boolean isConsistent(ProjectedColumn colReq, ColumnMetadata readCol) {
+    if (readCol.isDynamic()) {
+
+      // Don't know the type. This is a name-only probe.
+      return true;
+    }
+
+    // If the projection is map-like, but the proposed concrete column
+    // is not map-like, then the columns are not compatible.
+    if (colReq.isMap() && !(readCol.isMap() || readCol.isDict() || readCol.isVariant())) {
+      return false;
+    }
+
+    // If the projection is array-like, but the proposed concrete column
+    // is not map-like, or does not have at least as many dimensions as the
+    // projection, then the column is not compatible.
+    if (colReq.isArray()) {
+      if (colReq.arrayDims() == 1) {
+        return readCol.isArray() || readCol.isDict() || readCol.isVariant();
+      } else {
+        return readCol.type() == MinorType.LIST || readCol.isDict() || readCol.isVariant();
+      }
+    }
+    return true;
+  }
+
+  /**
+   * Perform the column-level projection as described in
+   * {@link #isConsistent(RequestedColumn, ColumnMetadata)}, and raise a
+   * {@code UserException} if the column is not consistent with projection.
+   *
+   * @param colReq the column-level projection description
+   * @param actual metadata for the column which the reader has actually
+   * produced
+   * @param errorContext additional error context to pass along in the
+   * exception
+   * @throws UserException if the read column is not consistent with the
+   * projection description for the column
+   */
+  public static void verifyCompatibility(ProjectedColumn colReq, ColumnMetadata actual,
+      String source, CustomErrorContext errorContext) {
+    if (!isConsistent(colReq, actual)) {
+      throw UserException.validationError()
+        .message(source + " column type not compatible with projection specification")
+        .addContext("Projected column", colReq.projectString())
+        .addContext(source + " column", actual.columnString())
+        .addContext(errorContext)
+        .build(logger);
+    }
+  }
+
+  public static void verifyConsistency(ColumnMetadata existing,
+      ColumnMetadata revised, String source,
+      CustomErrorContext errorContext) {
+    if (existing.isDynamic() || revised.isDynamic()) {
+      return;
+    }
+    if (existing.type() != revised.type() ||
+        existing.mode() != revised.mode()) {
+     throw UserException.validationError()
+       .message("Scan and " + source + " column type conflict")
+       .addContext("Scan column", existing.columnString())
+       .addContext(source + " column", revised.columnString())
+       .addContext(errorContext)
+       .build(logger);
+    }
+  }
+
+  public static void verifyProjection(ColumnMetadata existing,
+      ColumnMetadata revised, String source,
+      CustomErrorContext errorContext) {
+    if (existing instanceof ProjectedColumn) {
+      verifyCompatibility((ProjectedColumn) existing, revised, source, errorContext);
+    } else {
+      verifyConsistency(existing, revised, source, errorContext);
+    }
+  }
+
+  public static void mergeColProperties(ColumnMetadata existing, ColumnMetadata revised) {
+    mergeProperties(existing, revised);
+    if (existing.isMap() && revised.isMap()) {
+      mergeProperties(existing.tupleSchema(), revised.tupleSchema());
+    }
+  }
+
+  public static void mergeProperties(Propertied existing, Propertied revised) {
+    if (!revised.hasProperties()) {
+      return;
+    }
+    existing.properties().putAll(revised.properties());
+  }
+
+  public static boolean isStrict(TupleMetadata schema) {
+    return schema.booleanProperty(TupleMetadata.IS_STRICT_SCHEMA_PROP);
+  }
+
+  public static void markStrict(TupleMetadata schema) {
+    schema.setBooleanProperty(TupleMetadata.IS_STRICT_SCHEMA_PROP, true);
+  }
+
+  public static String implicitColType(ColumnMetadata col) {
+    return col.property(ColumnMetadata.IMPLICIT_COL_TYPE);
+  }
+
+  public static boolean isImplicit(ColumnMetadata col) {
+    return implicitColType(col) != null;
+  }
+
+  public static void markImplicit(ColumnMetadata col, String value) {
+    col.setProperty(ColumnMetadata.IMPLICIT_COL_TYPE, value);
+  }
+
+  public static void markAsPartition(ColumnMetadata col, int level) {
+    markImplicit(col, ColumnMetadata.IMPLICIT_PARTITION_PREFIX + level);
+  }
+
+  public static void markExcludeFromWildcard(ColumnMetadata col) {
+    col.setBooleanProperty(ColumnMetadata.EXCLUDE_FROM_WILDCARD, true);
+  }
+
+  public static boolean isExcludedFromWildcard(ColumnMetadata col) {
+    return col.booleanProperty(ColumnMetadata.EXCLUDE_FROM_WILDCARD);
+  }
+
+  public static ScanProjectionParser.ProjectionParseResult projectAll() {
+    TupleMetadata projSet = new TupleSchema();
+    projSet.setProperty(ScanProjectionParser.PROJECTION_TYPE_PROP, ScanProjectionParser.PROJECT_ALL);
+    return new ScanProjectionParser.ProjectionParseResult(0, projSet);
+  }
+
+  public static void markProjectAll(ColumnMetadata col) {
+    Preconditions.checkArgument(col.isMap());
+    col.tupleSchema().setProperty(ScanProjectionParser.PROJECTION_TYPE_PROP, ScanProjectionParser.PROJECT_ALL);
+  }
+
+  public static ScanProjectionParser.ProjectionParseResult projectNone() {
+    TupleMetadata projSet = new TupleSchema();
+    projSet.setProperty(ScanProjectionParser.PROJECTION_TYPE_PROP, ScanProjectionParser.PROJECT_NONE);
+    return new ScanProjectionParser.ProjectionParseResult(-1, projSet);
+  }
+
+  public static boolean isProjectAll(TupleMetadata tuple) {
+    return ScanProjectionParser.PROJECT_ALL.equals(tuple.property(ScanProjectionParser.PROJECTION_TYPE_PROP));
+  }
+
+  public static boolean isProjectNone(TupleMetadata tuple) {
+    return ScanProjectionParser.PROJECT_NONE.equals(tuple.property(ScanProjectionParser.PROJECTION_TYPE_PROP));
+  }
+
+  public static void copyMapProperties(ProjectedColumn source,
+      ColumnMetadata dest) {
+    if (source != null && source.isMap()) {
+      Preconditions.checkArgument(dest.isMap());
+      SchemaUtils.copyProperties(source.tupleSchema(), dest.tupleSchema());
+    } else {
+      markProjectAll(dest);
+    }
+  }
+
+  static void copyProperties(TupleMetadata source,
+      TupleMetadata dest) {
+    String value = source.property(ScanProjectionParser.PROJECTION_TYPE_PROP);
+    if (value != null) {
+      dest.setProperty(ScanProjectionParser.PROJECTION_TYPE_PROP, value);
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/package-info.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/package-info.java
new file mode 100644
index 0000000..41a783a
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/v3/schema/package-info.java
@@ -0,0 +1,378 @@
+/*
+ * 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.
+ */
+
+/**
+ * Provides run-time semantic analysis of the projection list for the
+ * scan operator. The project list can include table columns and a
+ * variety of special columns. Requested columns can exist in the table,
+ * or may be "missing" with null values applied. The code here prepares
+ * a run-time projection plan based on the actual table schema.
+ * <p>
+ * Resolves a scan schema throughout the scan lifecycle. Schema resolution
+ * comes from a variety of sources. Resolution starts with preparing the
+ * schema for the first reader:
+ * <ul>
+ * <li>Project list (wildcard, empty, or explicit)</li>
+ * <li>Optional provided schema (strict or lenient)</li>
+ * <li>Implicit columns</li>
+ * <li>An "early" reader schema (one determined before reading any
+ * data.</li>
+ * </ul>
+ * The result is a <i>defined schema</i> which may include;
+ * <ul>
+ * <li>Dynamic columns: those from the project list where we know only
+ * the column name, but not its type.</li>
+ * <li>Resolved columns: implicit or provided columns where we know
+ * the name and type.</li>
+ * </ul>
+ * The schema itself can be one of two forms:
+ * <ul>
+ * <li>Open: meaning that the reader can add other columns. An open
+ * schema results from a wildcard projection. Since the wildcard can appear
+ * along with implicit columns, the schema can be open and have a set of
+ * columns. If a provided schema appears, then the provided schema is
+ * expanded here. If the schema is "lenient", then the reader can add
+ * additional columns as it discovers them.</li>
+ * <li>Closed: meaning that the reader cannot add additional columns.
+ * A closed schema results from an empty or explicit projection list. A closed
+ * schema also results from a wildcard projection and a strict schema.</li>
+ * </ul>
+ * <p>
+ * Internally, the schema may start as open (has a wildcard), but may transition
+ * to closed when processing a strict provided schema.
+ * <p>
+ * Once this class is complete, the scan can add columns only to an open schema.
+ * All such columns are inserted at the wildcard location. If the wildcard appears
+ * by itself, columns are appended. If the wildcard appears along with implicit columns,
+ * then the reader columns appear at the wildcard location, before the implicit columns.
+ * <p>
+ * Once we have the initial reader input schema, we can then further refine
+ * the schema with:
+ * <ul>
+ * <li>The reader "output" schema: the columns actually read by the
+ * reader.</li>
+ * <li>The set of "missing" columns: those projected, but which the reader did
+ * not provide. We must make up a type for missing columns (and hope we guess
+ * correctly.) In fact, the purpose of the provided (and possibly early reader)
+ * schema is to avoid the need to guess.</li>
+ * </ul>
+ *
+ * <h4>Implicit (Wildcard) Projection</h4>
+ *
+ * A query can contain a wildcard ({@code *}). In this case, the set of columns is
+ * driven by the reader. Each scan might drive one, two or many readers. In an ideal
+ * world, every reader would produce the same schema. In the real world, files tend
+ * the evolve: early files have three columns, later files have five. In this case
+ * some readers will produce one schema, other readers another. Much of the complexity
+ * of Drill comes from this simple fact that Drill is a SQL engine that requires a
+ * single schema for all rows, but Drill reads data sources which are free to return
+ * any schema that they want.
+ * <p>
+ * A wildcard projection starts by accepting the schema produced by the first reader.
+ * In "classic" mode, later readers can add columns (causing a schema change to be
+ * sent downstream), but cannot change the types of existing columns. The code
+ * here supports a "no schema change" mode in which the first reader discovers the
+ * schema, which is then fixed for all subsequent readers. This mode cannot, however
+ * prevent schema conflicts across scans running in different fragments.
+ *
+ * <h4>Explicit Projection</h4>
+ *
+ * Explicit projection provides the list of columns, but not their types.
+ * Example: SELECT a, b, c.
+ * <p>
+ * The projection list holds the columns
+ * as requested by the user in the {@code SELECT} clause of the query,
+ * in the order which columns appear in that clause, along with additional
+ * columns implied by other columns. The planner
+ * determines which columns to project. In Drill, projection is speculative:
+ * it is a list of names which the planner hopes will appear in the data
+ * files. The reader must make up columns (the infamous nullable INT) when
+ * it turns out that no such column exists. Else, the reader must figure out
+ * the data type for any columns that does exist.
+ * <p>
+ * An explicit projection starts with the requested set of columns,
+ * then looks in the table schema to find matches. Columns not in the project list
+ * are not projected (not written to vectors). The reader columns provide the types
+ * of the projected columns, "resolving" them to a concrete type.
+ * <p>
+ * An explicit projection may include columns that do not exist in
+ * the source schema. In this case, we fill in null columns for
+ * unmatched projections.
+ * <p>
+ * The challenge in this case is that Drill cannot know the type of missing columns;
+ * Drill can only guess. If a reader in Scan 1 guesses a type, but a reader in
+ * Scan 2 reads a column with a different type, then a schema conflict will
+ * occur downstream.
+ *
+ * <h4>Maps</h4>
+ *
+ * Maps introduce a large amount of additional complexity. First, maps appear
+ * in the project list as either:
+ * <ul>
+ * <li>A generic projection: just the name {@code m}, where {@code m} is a map.
+ * In this case, we project all members of the map. That is, the map itself
+ * is open in the above sense. Note that a map can be open even if the scan
+ * schema itself is closed. That is, if the projection list contains only
+ * {@code m}, the scan schema is closed, but the map is open (the reader will
+ * discover the fields that make up the map.)</li>
+ * <li>A specific projection: a list of map members: {@code m.x, m.y}. In this
+ * case, we know that the downstream Project operator will pull just those two
+ * members to the top level and discard the rest of the map. We can thus
+ * project just those two members in the scan. As a result, the map is closed
+ * in the above sense: any additional map members discovered by the reader will
+ * be unprojected.</li>
+ * <li>Hybrid: a projection list that includes both: {@code m, m.x}. Here, the
+ * generic projection takes precedence. If the specific projection includes
+ * qualifiers, {@code m, m.x[1]}, then that information is used to check the
+ * type of column {@code x}.</li>
+ * <li>Implied: in a wildcard projection, a column may turn out to be a map.
+ * In this case, the map is open when the schema itself is open. (Remember that
+ * a wildcard projection can result in a closed schema if paired with a strict
+ * provided schema.</li>
+ * </ul>
+ *
+ * <h4>Schema Definition</h4>
+ *
+ * This resolver is the first step in the scan schema process. The result is a
+ * (typically dynamic) <i>defined schema</i>. To understand this concept, it helps
+ * to compare Drill with other query engines. In most engines, the planner is
+ * responsible for working out the scan schema from table metadata, from the
+ * project list and so on. The scan is given a fully-defined schema which it
+ * must use.
+ * <p>
+ * Drill is unique in that it uses a <i>dynamic schema</i> with columns and/or types
+ * "to be named later." The scan must convert the dynamic schema into a concrete
+ * schema sent downstream. This class implements some of the steps in doing so.
+ * <p>
+ * The result of this class is a schema identical to a defined schema that a
+ * planner might produce. Since Drill is dynamic, the planner must be able to
+ * produce a dynamic schema of the form described above. If the planner has table
+ * metadata (here represented by a provided schema), then the planner could produce
+ * a concrete defined schema (all types are defined.) Or, with a lenient provided
+ * schema, the planner might produce a dynamic defined schema: one with some
+ * concrete columns, some dynamic (name-only) columns.
+ *
+ * <h4>Implicit Columns</h4>
+ *
+ * This class handles one additional source of schema information: implicit
+ * columns: those defined by Drill itself. Examples include {@code filename,
+ * dir0}, etc. Implicit columns are available (at present) only for the file
+ * storage plugin, but could be added for other storage plugins. The project list
+ * can contain the names of implicit columns. If the query contains a wildcard,
+ * then the project list may also contain implicit columns:
+ * {@code filename, *, dir0}.
+ * <p>
+ * Implicit columns are known to Drill, so Drill itself can provide type information
+ * for those columns, by an external implicit column parser. That parser locates
+ * implicit columns by name, marks the columns as implicit, and takes care of
+ * populating the columns at read time. We use a column property,
+ * {@code IMPLICIT_COL_TYPE}, to mark a column as implicit. Later the scan mechanism
+ * will omit such columns when preparing the <i>reader schema</i>.
+ * <p>
+ * If the planner were to provide a defined schema, then the planner would have
+ * parsed out the implicit columns, provided their types, and marked them as
+ * implicit. So, again, we see that this class produces, at scan time, the same
+ * defined schema that the planner might produce at plan time.
+ * <p>
+ * Because of the way we handle implicit columns, we can allow the provided
+ * schema to include them. The provided schema simply adds a column (with any
+ * name), and sets the {@code IMPLICIT_COL_TYPE} property to indicate which
+ * implicit column definition to use for that column. This is handy for allowing the
+ * implicit column to include partition directories as regular columns.
+ * <p>
+ * We now have a parsing flow for this package:
+ * <ul>
+ * <li>Projection list (so we know what to include)</li>
+ * <li>Provided schema (to add/mark columns as implicit)</li>
+ * <li>Implicit columns, which looks for only for a) columns tagged as
+ * implicit or b) dynamic columns (those not defined in the provided
+ * schema.</li>
+ * </ul>
+ * <p>
+ * Drill has long had a source of ambiguity: what happens if the reader has a column
+ * with the same name as an implicit column. In this flow, the ambiguity is resolved
+ * as follows:
+ * <ul>
+ * <li>If a provided schema has a column explicitly tagged as an implicit column,
+ * then that column is unambiguously an implicit column independent of name.</li>
+ * <li>If a provided schema has a column with the same name as an implicit column
+ * (the names can be changed by a system/session option), then the fact that the
+ * column is not marked as implicit unambiguously tells us that the column is not
+ * implicit, despite the name.</li>
+ * <li>If a column appears in the project list, but not in the provided schema,
+ * and that column matches the (effective) name of some implicit column, then
+ * the column is marked as implicit and is not passed to the reader. Further, the
+ * projection filter will mark that column as unprojected in the reader, even if
+ * the reader otherwise has a wildcard schema.</li>
+ * </ul>
+ *
+ * <h4>Projection</h4>
+ *
+ * In prior versions of the scan operator, projection tended to be quite simple:
+ * just check if a name appears in the project list. As we've seen from the above,
+ * projection is actually quite complex with the need to reuse type information
+ * where available, open and closed top-level and map schemas, the need to avoid
+ * projecting columns with the same name as implicit columns, etc.
+ * <p>
+ * The {@code ProjectionFilter} classes handle projection. As it turns out, this
+ * class must follow (variations of) the same rules when merging the provided
+ * schema with the projection list and so on. To ensure a single implementation
+ * of the complex projection rules, this class uses a projection filter when
+ * resolving the provided schema. The devil is in the details, knowing when
+ * a map is open or closed, enforcing consistency with known information, etc.
+ *
+ * <h4>Provided Schema</h4>
+ *
+ * With the advent of provided schema in Drill 1.16, the query plan can provide
+ * not just column names (dynamic columns) but also the data type (concrete
+ * columns.) In this case, the scan schema can resolve projected columns against
+ * the provided schema, rather than waiting for the reader schema. Readers can use
+ * the provided schema to choose a column type when the choice is ambiguous, or multiple
+ * choices are possible.
+ * <p>
+ * If the projection list is a wildcard, then the wildcard expands to include all
+ * columns from the provided schema, in the order of that schema. If the schema
+ * is strict, then the scan schema becomes fixed, as if an explicit projection list
+ * where used.
+ * <p>
+ * If the projection list is explicit, then each column is resolved against
+ * the provided schema. If the projection list includes a column not in the
+ * provided schema, then it falls to the reader (or missing columns mechanism)
+ * to resolve that particular column.
+ *
+ * <h4>Early Reader Schema</h4>
+ *
+ * Some readers can declare their schema before reading data. For example, a JDBC
+ * query gets back a row schema during the initial prepare step. In this case, the
+ * reader is said to be <i>early schema</i>. The reader indicates an early schema
+ * via its <i>schema negotiator</i>. The framework then uses this schema to resolve
+ * the dynamic columns in the scan schema. If all columns are resolved this way,
+ * then the scan can declare its own schema before reading any data.
+ * <p>
+ * An early reader schema can work with a provided schema. In this case, the early
+ * reader schema must declare the same column type as the provided schema.
+ * This is not a large obstacle: the provided schema should have originally come
+ * from the reader (or a description of the reader) so conflicts should not
+ * occur in normal operation.
+ *
+ * <h4>Reader Output Schema</h4>
+ *
+ * Once a reader loads a batch of data, it provides (via the
+ * {@code ResultSetLoader}) the reader's <i>output schema</i>: the set of columns
+ * actually read by the reader.
+ * <p>
+ * If the projection list contained a wildcard, then the reader output schema
+ * will determine the set of columns that replaces the wildcard. (That is, all reader
+ * columns are projected and the scan schema expands to reflect the actual columns.)
+ * <p>
+ * If the projection list is explicit (or made so by a strict provided schema),
+ * then the reader output schema must be a subset of the scan schema: it is an error
+ * for the reader to include extra columns as the scan mechanism won't know what to
+ * do with those vectors. The projection mechanism (see below) integrates with the
+ * {@code ResultSetLoader} to project only those columns needed; the others are
+ * given to the reader as "dummy" column writers: writers that accept, but discard
+ * their data.
+ * <p>
+ * Note the major difference between the early reader schema and the reader output
+ * schema. The early reader schema includes all the columns that the reader can read.
+ * The reader output schema includes only those columns that the reader actually read
+ * (as controlled by the projection filter.) For most readers (CSV, JSON, etc.), there
+ * is no early reader schema, there is only the reader output schema: the set of columns
+ * (modulo projection) that turned out to be in the data source.
+ *
+ * <h4>Projection</h4
+ *
+ * The projection list tells the reader which columns to read. In this mechanism,
+ * the projection list undergoes multiple transforms (expanding into a provided
+ * schema, identifying implicit columns, etc.) Further, as columns are resolved
+ * (via a provided schema, an earlier reader, etc.), the projection list can provide
+ * type information as well.
+ * <p>
+ * To handle this, projection is driven by the (evolving) scan schema. In fact, the
+ * schema mechanism uses the same projection implementation when applying the
+ * provided schema and early reader schema.
+ *
+ * <h4>Assembling the Output Schema and Batch</h4>
+ *
+ * The <i>scan output schema</i> consists of up to three parts:
+ * <ul>
+ * <li>Reader columns (the reader output schema)</li>
+ * <li>Missing columns (reader input columns which the reader does not
+ * actually provide.)</li>
+ * <li>Implicit columns.</li>
+ * </ul>
+ * Distinct mechanisms build each kind of schema. The reader builds the vectors
+ * for the reader schema. A missing column handler builds the missing columns
+ * (using provided or inferred types and values.) An implicit column manager
+ * fills in the implicit columns based on file information.
+ * <p>
+ * The scan schema tracker tracks all three schemas together to form the
+ * scan output schema. Tracking the combined schema ensures we preserve the
+ * user's requested project ordering. The reader manager builds the vectors
+ * using the above mechanisms, then merges the vectors (very easy to do in a
+ * columnar system) to produce the output batch which matches the scan schema.
+ *
+ * <h4>Architecture Overview</h4>
+ *
+ * <pre>
+ *                   Scan Plan
+ *                       |
+ *                       v
+ *               +--------------+
+ *               | Project List |
+ *               |    Parser    |
+ *               +--------------+
+ *                       |
+ *                       v
+ *                +-------------+
+ *                | Scan Schema |     +-------------------+
+ *                |   Tracker   | --->| Projection Filter |
+ *                +-------------+     +-------------------+
+ *                       |                  |
+ *                       v                  v
+ *  +------+      +------------+     +------------+      +-----------+
+ *  | File | ---> |   Reader   |---->| Result Set | ---> | Data File |
+ *  | Data |      |            |     |   Loader   | <--- |  Reader   |
+ *  +------+      +------------+     +------------+      +-----------+
+ *                       |                  |
+ *                       v                  |
+ *                +------------+    Reader  |
+ *                |   Reader   |    Schema  |
+ *                | Lifecycle  | <----------+
+ *                +------------+            |
+ *                       |                  |
+ *                       v                  |
+ *                  +---------+    Loaded   |
+ *                  | Output  |    Vectors  |
+ *                  | Builder | <-----------+
+ *                  +---------+
+ *                       |
+ *                       v
+ *                 Output Batch
+ * </pre>
+ *
+ * Omitted are the details of implicit and missing columns. The scan lifecycle
+ * (not shown) orchestrates the whole process.
+ * <p>
+ * The result is a scan schema which can start entirely dynamic (just a wildcard
+ * or list of column names), which is then resolved via a series of steps (some
+ * of which involve the real work of the scanner: reading data.) The bottom is
+ * the output: a full-resolved scan schema which exactly describes an output
+ * data batch.
+ */
+package org.apache.drill.exec.physical.impl.scan.v3.schema;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ColumnBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ColumnBuilder.java
index 2661898..291888d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ColumnBuilder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ColumnBuilder.java
@@ -24,6 +24,7 @@ import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.physical.resultSet.impl.ColumnState.PrimitiveColumnState;
 import org.apache.drill.exec.physical.resultSet.impl.ListState.ListVectorState;
+import org.apache.drill.exec.physical.resultSet.impl.ProjectionFilter.ProjResult;
 import org.apache.drill.exec.physical.resultSet.impl.RepeatedListState.RepeatedListColumnState;
 import org.apache.drill.exec.physical.resultSet.impl.RepeatedListState.RepeatedListVectorState;
 import org.apache.drill.exec.physical.resultSet.impl.SingleVectorState.OffsetVectorState;
@@ -128,7 +129,7 @@ public class ColumnBuilder {
   private ColumnState buildPrimitive(ContainerState parent, ColumnMetadata columnSchema) {
 
     final ValueVector vector;
-    if (parent.projection().isProjected(columnSchema) || allowCreation(parent)) {
+    if (parent.projection().projection(columnSchema).isProjected || allowCreation(parent)) {
 
       // Create the vector for the column.
       vector = parent.vectorCache().vectorFor(columnSchema.schema());
@@ -210,11 +211,11 @@ public class ColumnBuilder {
   private ColumnState buildSingleMap(ContainerState parent, ColumnMetadata columnSchema) {
 
     final ProjectionFilter projFilter = parent.projection();
-    final boolean isProjected = projFilter.isProjected(columnSchema);
+    final ProjResult projResult = projFilter.projection(columnSchema);
 
     final MapVector vector;
     final VectorState vectorState;
-    if (isProjected) {
+    if (projResult.isProjected) {
 
       // Don't get the map vector from the vector cache. Map vectors may
       // have content that varies from batch to batch. Only the leaf
@@ -229,19 +230,19 @@ public class ColumnBuilder {
     final TupleObjectWriter mapWriter = MapWriter.buildMap(columnSchema, vector, new ArrayList<>());
     final SingleMapState mapState = new SingleMapState(parent.loader(),
         parent.vectorCache().childCache(columnSchema.name()),
-        projFilter.mapProjection(isProjected, columnSchema.name()));
+        projResult.mapFilter);
     return new MapColumnState(mapState, mapWriter, vectorState, parent.isVersioned());
   }
 
   private ColumnState buildMapArray(ContainerState parent, ColumnMetadata columnSchema) {
 
     final ProjectionFilter projFilter = parent.projection();
-    final boolean isProjected = projFilter.isProjected(columnSchema);
+    final ProjResult projResult = projFilter.projection(columnSchema);
 
     // Create the map's offset vector.
     final RepeatedMapVector mapVector;
     final UInt4Vector offsetVector;
-    if (isProjected) {
+    if (projResult.isProjected) {
 
       // Creating the map vector will create its contained vectors if we
       // give it a materialized field with children. So, instead pass a clone
@@ -266,7 +267,7 @@ public class ColumnBuilder {
 
     // Wrap the offset vector in a vector state
     VectorState offsetVectorState;
-    if (!projFilter.isProjected(columnSchema)) {
+    if (!projResult.isProjected) {
       offsetVectorState = new NullVectorState();
     } else {
       offsetVectorState = new OffsetVectorState(
@@ -279,7 +280,7 @@ public class ColumnBuilder {
     // Assemble it all into the column state.
     final MapArrayState mapState = new MapArrayState(parent.loader(),
         parent.vectorCache().childCache(columnSchema.name()),
-        projFilter.mapProjection(isProjected, columnSchema.name()));
+        projResult.mapFilter);
     return new MapColumnState(mapState, writer, mapVectorState, parent.isVersioned());
   }
 
@@ -524,12 +525,12 @@ public class ColumnBuilder {
   private ColumnState buildDictArray(ContainerState parent, ColumnMetadata columnSchema) {
 
     final ProjectionFilter projFilter = parent.projection();
-    final boolean isProjected = projFilter.isProjected(columnSchema);
+    final ProjResult projResult = projFilter.projection(columnSchema);
 
     // Create the dict's offset vector.
     final RepeatedDictVector repeatedDictVector;
     final UInt4Vector offsetVector;
-    if (isProjected) {
+    if (projResult.isProjected) {
 
       // Creating the dict vector will create its contained vectors if we
       // give it a materialized field with children. So, instead pass a clone
@@ -559,7 +560,7 @@ public class ColumnBuilder {
 
     VectorState offsetVectorState;
     VectorState dictOffsetVectorState;
-    if (!projFilter.isProjected(columnSchema)) {
+    if (!projResult.isProjected) {
       offsetVectorState = new NullVectorState();
       dictOffsetVectorState = new NullVectorState();
     } else {
@@ -579,7 +580,7 @@ public class ColumnBuilder {
     // Assemble it all into the column state.
     final TupleState.DictArrayState dictArrayState = new TupleState.DictArrayState(parent.loader(),
         parent.vectorCache().childCache(columnSchema.name()),
-        projFilter.mapProjection(isProjected, columnSchema.name()));
+        projResult.mapFilter);
     return new TupleState.DictColumnState(
         dictArrayState, writer, mapVectorState, parent.isVersioned());
   }
@@ -587,12 +588,12 @@ public class ColumnBuilder {
   private ColumnState buildSingleDict(ContainerState parent, ColumnMetadata columnSchema) {
 
     final ProjectionFilter projFilter = parent.projection();
-    final boolean isProjected = projFilter.isProjected(columnSchema);
+    final ProjResult projResult = projFilter.projection(columnSchema);
 
     // Create the dict's offset vector.
     final DictVector dictVector;
     final UInt4Vector offsetVector;
-    if (isProjected) {
+    if (projResult.isProjected) {
 
       // Creating the dict vector will create its contained vectors if we
       // give it a materialized field with children. So, instead pass a clone
@@ -617,7 +618,7 @@ public class ColumnBuilder {
     // Wrap the offset vector in a vector state
 
     final VectorState offsetVectorState;
-    if (!projFilter.isProjected(columnSchema)) {
+    if (!projResult.isProjected) {
       offsetVectorState = new NullVectorState();
     } else {
       offsetVectorState = new OffsetVectorState(
@@ -629,7 +630,7 @@ public class ColumnBuilder {
 
     // Assemble it all into the column state.
     final SingleDictState dictArrayState = new SingleDictState(parent.loader(), parent.vectorCache().childCache(columnSchema.name()),
-        projFilter.mapProjection(isProjected, columnSchema.name()));
+        projResult.mapFilter);
     return new TupleState.DictColumnState(
         dictArrayState, writer, mapVectorState, parent.isVersioned());
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ColumnState.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ColumnState.java
index 62e17a3..6d67ebc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ColumnState.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ColumnState.java
@@ -240,7 +240,6 @@ public abstract class ColumnState {
    * for some previous row, depending on exactly when and where the overflow
    * occurs.
    */
-
   public void rollover() {
     assert state == State.NORMAL;
 
@@ -252,7 +251,6 @@ public abstract class ColumnState {
     // vector to double beyond the limit, but that will require a bit
     // of thought to get right -- and, of course, completely defeats
     // the purpose of limiting vector size to avoid memory fragmentation...
-
     if (loader.rowIndex() == 0) {
       throw UserException
         .memoryError("A single column value is larger than the maximum allowed size of 16 MB")
@@ -288,7 +286,6 @@ public abstract class ColumnState {
 
         // Remember that we have look-ahead values stashed away in the
         // backup vector.
-
         state = State.LOOK_AHEAD;
         break;
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ProjectionFilter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ProjectionFilter.java
index 37f77a6..0d97429 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ProjectionFilter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ProjectionFilter.java
@@ -19,7 +19,7 @@ package org.apache.drill.exec.physical.resultSet.impl;
 
 import org.apache.drill.common.exceptions.CustomErrorContext;
 import org.apache.drill.common.exceptions.UserException;
-import org.apache.drill.exec.physical.resultSet.project.Projections;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.SchemaUtils;
 import org.apache.drill.exec.physical.resultSet.project.RequestedTuple;
 import org.apache.drill.exec.physical.resultSet.project.RequestedTuple.TupleProjectionType;
 import org.apache.drill.exec.record.metadata.ColumnMetadata;
@@ -28,7 +28,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Projection filter used when adding columns to the result set loader.
  * Provides a variety of ways to filter columns: no filtering, filter
  * by (parsed) projection list, or filter by projection list and
  * provided schema. Enforces consistency of actual reader schema and
@@ -42,32 +41,78 @@ public interface ProjectionFilter {
 
   ProjectionFilter PROJECT_ALL = new ImplicitProjectionFilter(true);
   ProjectionFilter PROJECT_NONE = new ImplicitProjectionFilter(false);
+  ProjResult NOT_PROJECTED = new ProjResult(false, null, PROJECT_NONE);
+  ProjResult PROJECTED = new ProjResult(true, null, PROJECT_ALL);
 
-  boolean isProjected(String colName);
+  public static class ProjResult {
+    public final boolean isProjected;
+    public final ColumnMetadata projection;
+    public final ProjectionFilter mapFilter;
+
+    public ProjResult(boolean isProjected) {
+      this(isProjected, null, null);
+    }
+
+    public ProjResult(boolean isProjected, ColumnMetadata projection) {
+      this(isProjected, projection, null);
+    }
+
+    public ProjResult(boolean isProjected, ColumnMetadata projection, ProjectionFilter mapFilter) {
+      this.isProjected = isProjected;
+      this.projection = projection;
+      this.mapFilter = mapFilter;
+    }
+  }
 
-  boolean isProjected(ColumnMetadata columnSchema);
+  ProjResult projection(ColumnMetadata columnSchema);
 
-  ProjectionFilter mapProjection(boolean isColProjected, String colName);
+  boolean isProjected(String colName);
 
   boolean isEmpty();
 
-  public static ProjectionFilter filterFor(RequestedTuple tupleProj,
+  public static ProjectionFilter projectionFilter(RequestedTuple tupleProj,
       CustomErrorContext errorContext) {
-    if (tupleProj.type() == TupleProjectionType.ALL) {
-      return PROJECT_ALL;
-    } else {
-      return new DirectProjectionFilter(tupleProj, errorContext);
+    switch (tupleProj.type()) {
+      case ALL:
+        return PROJECT_ALL;
+      case NONE:
+        return PROJECT_NONE;
+      default:
+        return new DirectProjectionFilter(tupleProj, errorContext);
     }
   }
 
-  public static ProjectionFilter filterFor(RequestedTuple tupleProj,
+  public static ProjectionFilter providedSchemaFilter(RequestedTuple tupleProj,
       TupleMetadata providedSchema, CustomErrorContext errorContext) {
+    if (tupleProj.type() == TupleProjectionType.NONE) {
+      return PROJECT_NONE;
+    }
     if (providedSchema == null) {
-      return filterFor(tupleProj, errorContext);
+      return projectionFilter(tupleProj, errorContext);
+    }
+    boolean strict = SchemaUtils.isStrict(providedSchema);
+    if (providedSchema.isEmpty()) {
+      if (strict) {
+        return PROJECT_NONE;
+      } else {
+        return projectionFilter(tupleProj, errorContext);
+      }
     }
+    ProjectionFilter schemaFilter = strict ?
+        new SchemaProjectionFilter(providedSchema, errorContext) :
+        new TypeProjectionFilter(providedSchema, errorContext);
     return new CompoundProjectionFilter(
         new DirectProjectionFilter(tupleProj, errorContext),
-        new SchemaProjectionFilter(providedSchema, errorContext));
+        schemaFilter);
+  }
+
+  public static ProjectionFilter definedSchemaFilter(
+      TupleMetadata definedSchema, CustomErrorContext errorContext) {
+    if (definedSchema.isEmpty()) {
+      return PROJECT_NONE;
+    } else {
+      return new SchemaProjectionFilter(definedSchema, errorContext);
+    }
   }
 
   /**
@@ -83,18 +128,17 @@ public interface ProjectionFilter {
     }
 
     @Override
-    public boolean isProjected(String name) {
-      return projectAll;
-    }
-
-    @Override
-    public boolean isProjected(ColumnMetadata columnSchema) {
-      return projectAll ? !Projections.excludeFromWildcard(columnSchema) : false;
+    public ProjResult projection(ColumnMetadata col) {
+      if (SchemaUtils.isExcludedFromWildcard(col)) {
+        return NOT_PROJECTED;
+      } else {
+        return projectAll ? PROJECTED : NOT_PROJECTED;
+      }
     }
 
     @Override
-    public ProjectionFilter mapProjection(boolean isColProjected, String colName) {
-      return isColProjected ? this : PROJECT_NONE;
+    public boolean isProjected(String name) {
+      return projectAll;
     }
 
     @Override
@@ -118,100 +162,134 @@ public interface ProjectionFilter {
     }
 
     @Override
+    public ProjResult projection(ColumnMetadata col) {
+      if (projectionSet.enforceProjection(col, errorContext)) {
+        return new ProjResult(true, null,
+            projectionFilter(projectionSet.mapProjection(col.name()), errorContext));
+      } else {
+        return NOT_PROJECTED;
+      }
+    }
+
+    @Override
     public boolean isProjected(String colName) {
       return projectionSet.isProjected(colName);
     }
 
     @Override
-    public boolean isProjected(ColumnMetadata columnSchema) {
-      return projectionSet.enforceProjection(columnSchema, errorContext);
+    public boolean isEmpty() {
+      return projectionSet.isEmpty();
     }
+  }
 
-    @Override
-    public ProjectionFilter mapProjection(boolean isColProjected, String colName) {
-      return isColProjected ?
-        filterFor(projectionSet.mapProjection(colName), errorContext) :
-        PROJECT_NONE;
+  /**
+   * Schema-based projection.
+   */
+  public abstract static class BaseSchemaProjectionFilter implements ProjectionFilter {
+    protected final TupleMetadata schema;
+    protected final CustomErrorContext errorContext;
+
+    private BaseSchemaProjectionFilter(TupleMetadata schema, CustomErrorContext errorContext) {
+      this.schema = schema;
+      this.errorContext = errorContext;
+    }
+
+    protected void validateColumn(ColumnMetadata schemaCol, ColumnMetadata readerCol) {
+      if (schemaCol.isDynamic()) {
+        return;
+      }
+      if (schemaCol.type() != readerCol.type() ||
+          schemaCol.mode() != readerCol.mode()) {
+        throw UserException.validationError()
+          .message("Reader and scan column type conflict")
+          .addContext("Scan column", schemaCol.columnString())
+          .addContext("Reader column", readerCol.columnString())
+          .addContext(errorContext)
+          .build(logger);
+      }
+    }
+
+    protected void validateMap(ColumnMetadata schemaCol) {
+      if (!schemaCol.isMap()) {
+        throw UserException.validationError()
+          .message("Reader expected a map column, but the the schema column is not a map")
+          .addContext("Provided column", schemaCol.columnString())
+          .addContext("Reader column", schemaCol.name())
+          .addContext(errorContext)
+          .build(logger);
+      }
     }
 
     @Override
     public boolean isEmpty() {
-      return projectionSet.isEmpty();
+       return schema.isEmpty();
     }
   }
 
   /**
-   * Projection based on a provided schema. If the schema is strict, a reader column
-   * is projected only if that column appears in the provided schema. Non-strict
-   * schema allow additional reader columns.
+   * Projection based on a non-strict provided schema which enforces the type of known
+   * columns, but has no opinion about additional columns.
    * <p>
    * If the column is found, enforces that the reader schema has the same type and
    * mode as the provided column.
    */
-  public static class SchemaProjectionFilter implements ProjectionFilter {
-    private final TupleMetadata providedSchema;
-    private final CustomErrorContext errorContext;
-    private final boolean isStrict;
+  public static class TypeProjectionFilter extends BaseSchemaProjectionFilter {
 
-    public SchemaProjectionFilter(TupleMetadata providedSchema, CustomErrorContext errorContext) {
-      this(providedSchema,
-          providedSchema.booleanProperty(TupleMetadata.IS_STRICT_SCHEMA_PROP),
-          errorContext);
+    public TypeProjectionFilter(TupleMetadata providedSchema, CustomErrorContext errorContext) {
+      super(providedSchema, errorContext);
     }
 
-    private SchemaProjectionFilter(TupleMetadata providedSchema, boolean isStrict, CustomErrorContext errorContext) {
-      this.providedSchema = providedSchema;
-      this.errorContext = errorContext;
-      this.isStrict = isStrict;
+    @Override
+    public ProjResult projection(ColumnMetadata col) {
+      ColumnMetadata providedCol = schema.metadata(col.name());
+      if (providedCol == null) {
+        return PROJECTED;
+      } else {
+        validateColumn(providedCol, col);
+        if (providedCol.isMap()) {
+          return new ProjResult(true, providedCol,
+              new TypeProjectionFilter(providedCol.tupleSchema(), errorContext));
+        } else {
+          return new ProjResult(true, providedCol);
+        }
+      }
     }
 
     @Override
     public boolean isProjected(String name) {
-      ColumnMetadata providedCol = providedSchema.metadata(name);
-      return providedCol != null || !isStrict;
+      return true;
     }
+  }
 
-    @Override
-    public boolean isProjected(ColumnMetadata columnSchema) {
-      ColumnMetadata providedCol = providedSchema.metadata(columnSchema.name());
-      if (providedCol == null) {
-        return !isStrict;
-      }
-      if (providedCol.type() != columnSchema.type() ||
-          providedCol.mode() != columnSchema.mode()) {
-        throw UserException.validationError()
-          .message("Reader and provided column type mismatch")
-          .addContext("Provided column", providedCol.columnString())
-          .addContext("Reader column", columnSchema.columnString())
-          .addContext(errorContext)
-          .build(logger);
-      }
-      return true;
+  /**
+   * Projection filter in which a schema exactly defines the set of allowed
+   * columns, and their types.
+   */
+  public static class SchemaProjectionFilter extends BaseSchemaProjectionFilter {
+
+    public SchemaProjectionFilter(TupleMetadata definedSchema, CustomErrorContext errorContext) {
+      super(definedSchema, errorContext);
     }
 
     @Override
-    public ProjectionFilter mapProjection(boolean isColProjected, String colName) {
-      if (!isColProjected) {
-        return PROJECT_NONE;
-      }
-      ColumnMetadata providedCol = providedSchema.metadata(colName);
+    public ProjResult projection(ColumnMetadata col) {
+      ColumnMetadata providedCol = schema.metadata(col.name());
       if (providedCol == null) {
-        return PROJECT_ALL;
-      }
-      if (!providedCol.isMap()) {
-        throw UserException.validationError()
-          .message("Reader expected a map column, but the the provided column is not a map")
-          .addContext("Provided column", providedCol.columnString())
-          .addContext("Reader column", colName)
-          .addContext(errorContext)
-          .build(logger);
+        return NOT_PROJECTED;
+      } else {
+        validateColumn(providedCol, col);
+        if (providedCol.isMap()) {
+          return new ProjResult(true, providedCol,
+              new SchemaProjectionFilter(providedCol.tupleSchema(), errorContext));
+        } else {
+          return new ProjResult(true, providedCol);
+        }
       }
-      return new SchemaProjectionFilter(providedCol.tupleSchema(), isStrict, errorContext);
     }
 
     @Override
-    public boolean isEmpty() {
-       return providedSchema.isEmpty();
+    public boolean isProjected(String name) {
+      return schema.metadata(name) != null;
     }
   }
 
@@ -228,37 +306,35 @@ public interface ProjectionFilter {
     }
 
     @Override
-    public boolean isProjected(String name) {
-      return filter1.isProjected(name) && filter2.isProjected(name);
-    }
-
-    @Override
-    public boolean isProjected(ColumnMetadata columnSchema) {
-      return filter1.isProjected(columnSchema) && filter2.isProjected(columnSchema);
-    }
-
-    @Override
-    public ProjectionFilter mapProjection(boolean isColProjected, String colName) {
-      ProjectionFilter childFilter1 = filter1.mapProjection(isColProjected, colName);
-      ProjectionFilter childFilter2 = filter2.mapProjection(isColProjected, colName);
-      if (childFilter1 == PROJECT_ALL) {
-        return childFilter2;
+    public ProjResult projection(ColumnMetadata col) {
+      ProjResult result1 = filter1.projection(col);
+      ProjResult result2 = filter2.projection(col);
+      if (!result1.isProjected || !result2.isProjected) {
+        return NOT_PROJECTED;
       }
-      if (childFilter1 == PROJECT_NONE) {
-        return childFilter1;
+      if (result1.mapFilter == null && result2.mapFilter == null) {
+        return result1;
       }
-      if (childFilter2 == PROJECT_ALL) {
-        return childFilter1;
+      if (result1.mapFilter == PROJECT_ALL) {
+        return result2;
       }
-      if (childFilter2 == PROJECT_NONE) {
-        return childFilter2;
+      if (result2.mapFilter == PROJECT_ALL) {
+        return result1;
       }
-      return new CompoundProjectionFilter(childFilter1, childFilter2);
+
+      return new ProjResult(true,
+          result1.projection == null ? result2.projection : result1.projection,
+          new CompoundProjectionFilter(result1.mapFilter, result2.mapFilter));
+    }
+
+    @Override
+    public boolean isProjected(String name) {
+      return filter1.isProjected(name) && filter2.isProjected(name);
     }
 
     @Override
     public boolean isEmpty() {
-      return filter1.isEmpty() && filter2.isEmpty();
+      return filter1.isEmpty() || filter2.isEmpty();
     }
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ResultSetLoaderImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ResultSetLoaderImpl.java
index cee582b..5510e4d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ResultSetLoaderImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ResultSetLoaderImpl.java
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.physical.resultSet.impl;
 
 import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.EmptyErrorContext;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
@@ -25,6 +26,7 @@ import org.apache.drill.exec.physical.resultSet.ResultVectorCache;
 import org.apache.drill.exec.physical.resultSet.RowSetLoader;
 import org.apache.drill.exec.physical.resultSet.impl.TupleState.RowState;
 import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.accessor.impl.HierarchicalFormatter;
@@ -72,14 +74,21 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
       vectorCache = builder.vectorCache;
       schema = builder.readerSchema;
       maxBatchSize = builder.maxBatchSize;
-      errorContext = builder.errorContext;
+      errorContext = builder.errorContext == null
+          ? EmptyErrorContext.INSTANCE : builder.errorContext;
       if (builder.projectionFilter != null) {
         projectionSet = builder.projectionFilter;
       } else if (builder.projectionSet != null) {
-        projectionSet = ProjectionFilter.filterFor(builder.projectionSet, errorContext);
+        projectionSet = ProjectionFilter.projectionFilter(builder.projectionSet, errorContext);
       } else {
         projectionSet = ProjectionFilter.PROJECT_ALL;
       }
+      if (schema != null && MetadataUtils.hasDynamicColumns(schema)) {
+        throw UserException.validationError()
+          .message("Reader input schema must not contain dynamic columns")
+          .addContext(errorContext)
+          .build(logger);
+      }
     }
 
     public void dump(HierarchicalFormatter format) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/ImpliedTupleRequest.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/ImpliedTupleRequest.java
index 09fc512..94ef8b6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/ImpliedTupleRequest.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/ImpliedTupleRequest.java
@@ -48,6 +48,12 @@ public class ImpliedTupleRequest implements RequestedTuple {
   }
 
   @Override
+  public int size() { return 0; }
+
+  @Override
+  public RequestedColumn get(int i) { return null; }
+
+  @Override
   public RequestedColumn get(String colName) { return null; }
 
   @Override
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/RequestedTuple.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/RequestedTuple.java
index 8d621a7..31c9505 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/RequestedTuple.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/RequestedTuple.java
@@ -59,6 +59,8 @@ public interface RequestedTuple {
   }
 
   TupleProjectionType type();
+  int size();
+  RequestedColumn get(int i);
   RequestedColumn get(String colName);
   boolean isProjected(String colName);
   boolean isProjected(ColumnMetadata columnSchema);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/RequestedTupleImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/RequestedTupleImpl.java
index c871e86..056d361 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/RequestedTupleImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/RequestedTupleImpl.java
@@ -112,6 +112,14 @@ public class RequestedTupleImpl implements RequestedTuple {
   }
 
   @Override
+  public int size() { return projection.count(); }
+
+  @Override
+  public RequestedColumn get(int i) {
+    return projection.get(i);
+  }
+
+  @Override
   public RequestedColumn get(String colName) {
     return projection.get(colName.toLowerCase());
   }
@@ -174,6 +182,7 @@ public class RequestedTupleImpl implements RequestedTuple {
         !Projections.excludeFromWildcard(columnSchema) : true;
   }
 
+  @Override
   public boolean enforceProjection(ColumnMetadata columnSchema, CustomErrorContext errorContext) {
     if (projectionType == TupleProjectionType.ALL) {
       return true;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ColumnExplorer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ColumnExplorer.java
index 4fd7abe..0467abc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ColumnExplorer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ColumnExplorer.java
@@ -32,6 +32,7 @@ import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.map.CaseInsensitiveMap;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.metastore.ColumnNamesOptions;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.apache.drill.exec.server.options.OptionManager;
 import org.apache.drill.exec.server.options.OptionValue;
 import org.apache.drill.exec.store.dfs.FileSelection;
@@ -97,7 +98,7 @@ public class ColumnExplorer {
     Map<String, ImplicitFileColumns> map = CaseInsensitiveMap.newHashMap();
     for (ImplicitFileColumns e : ImplicitFileColumns.values()) {
       OptionValue optionValue;
-      if ((optionValue = optionManager.getOption(e.name)) != null) {
+      if ((optionValue = optionManager.getOption(e.optionName)) != null) {
         map.put(optionValue.string_val, e);
       }
     }
@@ -130,7 +131,7 @@ public class ColumnExplorer {
     List<String> implicitColumns = Lists.newArrayList();
     for (ImplicitFileColumns e : ImplicitFileColumns.values()) {
       OptionValue optionValue;
-      if ((optionValue = schemaConfig.getOption(e.name)) != null) {
+      if ((optionValue = schemaConfig.getOption(e.optionName)) != null) {
         implicitColumns.add(optionValue.string_val);
       }
     }
@@ -495,7 +496,7 @@ public class ColumnExplorer {
     /**
      * Fully qualified name, contains full path to file and file name
      */
-    FQN (ExecConstants.IMPLICIT_FQN_COLUMN_LABEL) {
+    FQN (ExecConstants.IMPLICIT_FQN_COLUMN_LABEL, ColumnMetadata.IMPLICIT_FQN) {
       @Override
       public String getValue(Path path) {
         return path.toUri().getPath();
@@ -505,7 +506,7 @@ public class ColumnExplorer {
     /**
      * Full path to file without file name
      */
-    FILEPATH (ExecConstants.IMPLICIT_FILEPATH_COLUMN_LABEL) {
+    FILEPATH (ExecConstants.IMPLICIT_FILEPATH_COLUMN_LABEL, ColumnMetadata.IMPLICIT_FILEPATH) {
       @Override
       public String getValue(Path path) {
         return path.getParent().toUri().getPath();
@@ -515,7 +516,7 @@ public class ColumnExplorer {
     /**
      * File name with extension without path
      */
-    FILENAME (ExecConstants.IMPLICIT_FILENAME_COLUMN_LABEL) {
+    FILENAME (ExecConstants.IMPLICIT_FILENAME_COLUMN_LABEL, ColumnMetadata.IMPLICIT_FILENAME) {
       @Override
       public String getValue(Path path) {
         return path.getName();
@@ -525,21 +526,36 @@ public class ColumnExplorer {
     /**
      * File suffix (without dot at the beginning)
      */
-    SUFFIX (ExecConstants.IMPLICIT_SUFFIX_COLUMN_LABEL) {
+    SUFFIX (ExecConstants.IMPLICIT_SUFFIX_COLUMN_LABEL, ColumnMetadata.IMPLICIT_SUFFIX) {
       @Override
       public String getValue(Path path) {
         return Files.getFileExtension(path.getName());
       }
     };
 
-    String name;
+    String optionName;
+    String propValue;
 
-    ImplicitFileColumns(String name) {
-      this.name = name;
-    }
+    ImplicitFileColumns(String optionName, String propValue) {
+      this.optionName = optionName;
+      this.propValue = propValue;
+     }
 
+    /**
+     * The name of the session/system option that gives the effective
+     * name of this implicit column when parsing columns by name.
+     */
     @Override
-    public String optionName() { return name; }
+    public String optionName() { return optionName; }
+
+    /**
+     * The name of the column property that indicates the implicit
+     * column type when using a provided schema. The property value
+     * lives in a name space separate from column names and so is
+     * fixed: it remains the same independent of system/session
+     * options.
+     */
+    public String propertyValue() { return propValue; }
 
     /**
      * Using file path calculates value for each implicit file column
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/project/TestNullColumnLoader.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/project/TestNullColumnLoader.java
index cf63b43..882e216 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/project/TestNullColumnLoader.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/project/TestNullColumnLoader.java
@@ -51,7 +51,6 @@ import org.junit.experimental.categories.Category;
  * can create the classic nullable Int null column, or one of
  * any other type and mode.
  */
-
 @Category(RowSetTests.class)
 public class TestNullColumnLoader extends SubOperatorTest {
 
@@ -67,7 +66,6 @@ public class TestNullColumnLoader extends SubOperatorTest {
 
     // For this test, we don't need the projection, so just
     // set it to null.
-
     return new ResolvedNullColumn(name, nullType, defaultValue, null, 0);
   }
 
@@ -76,7 +74,6 @@ public class TestNullColumnLoader extends SubOperatorTest {
    * cache. Specify no column type, the special NULL type, or a
    * predefined type. Output types should be set accordingly.
    */
-
   @Test
   public void testBasics() {
 
@@ -91,11 +88,9 @@ public class TestNullColumnLoader extends SubOperatorTest {
     final NullColumnLoader staticLoader = new NullColumnLoader(cache, defns, null, false);
 
     // Create a batch
-
     final VectorContainer output = staticLoader.load(2);
 
     // Verify values and types
-
     final TupleMetadata expectedSchema = new SchemaBuilder()
         .add("unspecified", NullColumnLoader.DEFAULT_NULL_TYPE)
         .add("nullType", NullColumnLoader.DEFAULT_NULL_TYPE)
@@ -118,7 +113,6 @@ public class TestNullColumnLoader extends SubOperatorTest {
    * column is ever INT (nullable or otherwise) and we want our null
    * columns to be (non-nullable) VARCHAR.
    */
-
   @Test
   public void testCustomNullType() {
 
@@ -131,7 +125,6 @@ public class TestNullColumnLoader extends SubOperatorTest {
 
     // Null required is an oxymoron, so is not tested.
     // Null type array does not make sense, so is not tested.
-
     final ResultVectorCache cache = new NullResultVectorCacheImpl(fixture.allocator());
     final MajorType nullType = MajorType.newBuilder()
         .setMinorType(MinorType.VARCHAR)
@@ -140,11 +133,9 @@ public class TestNullColumnLoader extends SubOperatorTest {
     final NullColumnLoader staticLoader = new NullColumnLoader(cache, defns, nullType, false);
 
     // Create a batch
-
     final VectorContainer output = staticLoader.load(2);
 
     // Verify values and types
-
     final TupleMetadata expectedSchema = new SchemaBuilder()
         .add("unspecified", nullType)
         .add("nullType", nullType)
@@ -163,7 +154,6 @@ public class TestNullColumnLoader extends SubOperatorTest {
    * Default values are only allowed for required "null" columns. For
    * nullable columns, NULL is already the default.
    */
-
   @Test
   public void testDefaultValue() {
 
@@ -177,11 +167,9 @@ public class TestNullColumnLoader extends SubOperatorTest {
     final NullColumnLoader staticLoader = new NullColumnLoader(cache, defns, nullType, false);
 
     // Create a batch
-
     final VectorContainer output = staticLoader.load(2);
 
     // Verify values and types
-
     final TupleMetadata expectedSchema = new SchemaBuilder()
         .add("int", MinorType.INT)
         .add("str", MinorType.VARCHAR)
@@ -210,7 +198,6 @@ public class TestNullColumnLoader extends SubOperatorTest {
    * This test ensures that the null column mechanism looks in that
    * vector cache when asked to create a nullable column.
    */
-
   @Test
   public void testCachedTypesMapToNullable() {
 
@@ -221,28 +208,23 @@ public class TestNullColumnLoader extends SubOperatorTest {
     defns.add(makeNullCol("unk"));
 
     // Populate the cache with a column of each mode.
-
     final ResultVectorCacheImpl cache = new ResultVectorCacheImpl(fixture.allocator());
     cache.vectorFor(SchemaBuilder.columnSchema("req", MinorType.FLOAT8, DataMode.REQUIRED));
     final ValueVector opt = cache.vectorFor(SchemaBuilder.columnSchema("opt", MinorType.FLOAT8, DataMode.OPTIONAL));
     final ValueVector rep = cache.vectorFor(SchemaBuilder.columnSchema("rep", MinorType.FLOAT8, DataMode.REPEATED));
 
     // Use nullable Varchar for unknown null columns.
-
     final MajorType nullType = Types.optional(MinorType.VARCHAR);
     final NullColumnLoader staticLoader = new NullColumnLoader(cache, defns, nullType, false);
 
     // Create a batch
-
     final VectorContainer output = staticLoader.load(2);
 
     // Verify vectors are reused
-
     assertSame(opt, output.getValueVector(1).getValueVector());
     assertSame(rep, output.getValueVector(2).getValueVector());
 
     // Verify values and types
-
     final TupleMetadata expectedSchema = new SchemaBuilder()
         .addNullable("req", MinorType.FLOAT8)
         .addNullable("opt", MinorType.FLOAT8)
@@ -265,7 +247,6 @@ public class TestNullColumnLoader extends SubOperatorTest {
    * empty values (zero or "") -- if the scan operator feels doing so would
    * be helpful.
    */
-
   @Test
   public void testCachedTypesAllowRequired() {
 
@@ -276,28 +257,23 @@ public class TestNullColumnLoader extends SubOperatorTest {
     defns.add(makeNullCol("unk"));
 
     // Populate the cache with a column of each mode.
-
     final ResultVectorCacheImpl cache = new ResultVectorCacheImpl(fixture.allocator());
     cache.vectorFor(SchemaBuilder.columnSchema("req", MinorType.FLOAT8, DataMode.REQUIRED));
     final ValueVector opt = cache.vectorFor(SchemaBuilder.columnSchema("opt", MinorType.FLOAT8, DataMode.OPTIONAL));
     final ValueVector rep = cache.vectorFor(SchemaBuilder.columnSchema("rep", MinorType.FLOAT8, DataMode.REPEATED));
 
     // Use nullable Varchar for unknown null columns.
-
     final MajorType nullType = Types.optional(MinorType.VARCHAR);
     final NullColumnLoader staticLoader = new NullColumnLoader(cache, defns, nullType, true);
 
     // Create a batch
-
     final VectorContainer output = staticLoader.load(2);
 
     // Verify vectors are reused
-
     assertSame(opt, output.getValueVector(1).getValueVector());
     assertSame(rep, output.getValueVector(2).getValueVector());
 
     // Verify values and types
-
     final TupleMetadata expectedSchema = new SchemaBuilder()
         .add("req", MinorType.FLOAT8)
         .addNullable("opt", MinorType.FLOAT8)
@@ -319,7 +295,6 @@ public class TestNullColumnLoader extends SubOperatorTest {
    * to pull in the null columns which the null column loader has
    * created.
    */
-
   @Test
   public void testNullColumnBuilder() {
 
@@ -334,11 +309,9 @@ public class TestNullColumnLoader extends SubOperatorTest {
     builder.build(cache);
 
     // Create a batch
-
     builder.load(2);
 
     // Verify values and types
-
     final TupleMetadata expectedSchema = new SchemaBuilder()
         .add("unspecified", NullColumnLoader.DEFAULT_NULL_TYPE)
         .add("nullType", NullColumnLoader.DEFAULT_NULL_TYPE)
@@ -363,7 +336,6 @@ public class TestNullColumnLoader extends SubOperatorTest {
   public void testNullColumnBuilderWithSchema() {
 
     // Note: upper case names in schema, lower case in "projection" list
-
     final TupleMetadata outputSchema = new SchemaBuilder()
         .add("IntReq", MinorType.INT)
         .add("StrReq", MinorType.VARCHAR)
@@ -390,11 +362,9 @@ public class TestNullColumnLoader extends SubOperatorTest {
     builder.build(cache);
 
     // Create a batch
-
     builder.load(2);
 
     // Verify values and types
-
     final TupleMetadata expectedSchema = new SchemaBuilder()
         .add("strReq", MinorType.VARCHAR)
         .addNullable("strOpt", MinorType.VARCHAR)
@@ -431,7 +401,6 @@ public class TestNullColumnLoader extends SubOperatorTest {
   public void testSchemaWithConflicts() {
 
     // Note: upper case names in schema, lower case in "projection" list
-
     final TupleMetadata outputSchema = new SchemaBuilder()
         .add("IntReq", MinorType.INT)
         .add("StrReq", MinorType.VARCHAR) // No default
@@ -457,11 +426,9 @@ public class TestNullColumnLoader extends SubOperatorTest {
     builder.build(cache);
 
     // Create a batch
-
     builder.load(2);
 
     // Verify values and types
-
     final TupleMetadata expectedSchema = new SchemaBuilder()
         .addNullable("strReq", MinorType.VARCHAR)
         .addNullable("strOpt", MinorType.VARCHAR)
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/project/TestScanLevelProjection.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/project/TestScanLevelProjection.java
index 1e7a9a3..f4cee33 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/project/TestScanLevelProjection.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/project/TestScanLevelProjection.java
@@ -24,7 +24,7 @@ import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-import org.apache.drill.categories.RowSetTests;
+import org.apache.drill.categories.EvfTests;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos.MinorType;
@@ -47,9 +47,13 @@ import org.junit.experimental.categories.Category;
  * Test the level of projection done at the level of the scan as a whole;
  * before knowledge of table "implicit" columns or the specific table schema.
  */
-@Category(RowSetTests.class)
+@Category(EvfTests.class)
 public class TestScanLevelProjection extends SubOperatorTest {
 
+  private boolean isProjected(ProjectionFilter filter, ColumnMetadata col) {
+    return filter.projection(col).isProjected;
+  }
+
   /**
    * Basic test: select a set of columns (a, b, c) when the
    * data source has an early schema of (a, c, d). (a, c) are
@@ -96,10 +100,8 @@ public class TestScanLevelProjection extends SubOperatorTest {
         .buildSchema();
 
     ProjectionFilter projSet = scanProj.readerProjection();
-    assertTrue(projSet.isProjected("a"));
-    assertTrue(projSet.isProjected(readerSchema.metadata("a")));
-    assertFalse(projSet.isProjected("d"));
-    assertFalse(projSet.isProjected(readerSchema.metadata("d")));
+    assertTrue(isProjected(projSet, readerSchema.metadata("a")));
+    assertFalse(isProjected(projSet, readerSchema.metadata("d")));
   }
 
   /**
@@ -161,12 +163,9 @@ public class TestScanLevelProjection extends SubOperatorTest {
     // projection type is used here for testing; should not be used by
     // an actual reader.
     ProjectionFilter projSet = scanProj.readerProjection();
-    assertTrue(projSet.isProjected("a"));
-    assertTrue(projSet.isProjected(readerSchema.metadata("a")));
-    assertTrue(projSet.isProjected("c"));
-    assertTrue(projSet.isProjected(readerSchema.metadata("c")));
-    assertFalse(projSet.isProjected("d"));
-    assertFalse(projSet.isProjected(readerSchema.metadata("d")));
+    assertTrue(isProjected(projSet, readerSchema.metadata("a")));
+    assertTrue(isProjected(projSet, readerSchema.metadata("c")));
+    assertFalse(isProjected(projSet, readerSchema.metadata("d")));
   }
 
   /**
@@ -209,10 +208,8 @@ public class TestScanLevelProjection extends SubOperatorTest {
         .buildSchema();
 
     ProjectionFilter projSet = scanProj.readerProjection();
-    assertTrue(projSet.isProjected("a"));
-    assertTrue(projSet.isProjected(readerSchema.metadata("a")));
-    assertFalse(projSet.isProjected("c"));
-    assertFalse(projSet.isProjected(readerSchema.metadata("c")));
+    assertTrue(isProjected(projSet, readerSchema.metadata("a")));
+    assertFalse(isProjected(projSet, readerSchema.metadata("c")));
   }
 
   /**
@@ -252,8 +249,8 @@ public class TestScanLevelProjection extends SubOperatorTest {
         .buildSchema();
 
     ProjectionFilter projSet = scanProj.readerProjection();
-    assertTrue(projSet.isProjected(readerSchema.metadata("a")));
-    assertTrue(projSet.isProjected(readerSchema.metadata("c")));
+    assertTrue(isProjected(projSet, readerSchema.metadata("a")));
+    assertTrue(isProjected(projSet, readerSchema.metadata("c")));
   }
 
   /**
@@ -280,8 +277,7 @@ public class TestScanLevelProjection extends SubOperatorTest {
         .buildSchema();
 
     ProjectionFilter projSet = scanProj.readerProjection();
-    assertFalse(projSet.isProjected("a"));
-    assertFalse(projSet.isProjected(readerSchema.metadata("a")));
+    assertFalse(isProjected(projSet, readerSchema.metadata("a")));
   }
 
   /**
@@ -314,8 +310,8 @@ public class TestScanLevelProjection extends SubOperatorTest {
         .buildSchema();
 
     ProjectionFilter projSet = scanProj.readerProjection();
-    assertTrue(projSet.isProjected(readerSchema.metadata("a")));
-    assertTrue(projSet.isProjected(readerSchema.metadata("c")));
+    assertTrue(isProjected(projSet, readerSchema.metadata("a")));
+    assertTrue(isProjected(projSet, readerSchema.metadata("c")));
   }
 
   /**
@@ -389,10 +385,8 @@ public class TestScanLevelProjection extends SubOperatorTest {
     assertSame(providedSchema.metadata("b"), ((UnresolvedColumn) bCol).metadata());
 
     ProjectionFilter projSet = scanProj.readerProjection();
-    assertTrue(projSet.isProjected("a"));
-    assertTrue(projSet.isProjected(providedSchema.metadata("a")));
-    assertTrue(projSet.isProjected("b"));
-    assertTrue(projSet.isProjected(providedSchema.metadata("b")));
+    assertTrue(isProjected(projSet, providedSchema.metadata("a")));
+    assertTrue(isProjected(projSet, providedSchema.metadata("b")));
   }
 
   @Test
@@ -451,9 +445,9 @@ public class TestScanLevelProjection extends SubOperatorTest {
         .buildSchema();
 
     ProjectionFilter projSet = scanProj.readerProjection();
-    assertTrue(projSet.isProjected(readerSchema.metadata("a")));
-    assertTrue(projSet.isProjected(readerSchema.metadata("b")));
-    assertFalse(projSet.isProjected(readerSchema.metadata("c")));
+    assertTrue(isProjected(projSet, readerSchema.metadata("a")));
+    assertTrue(isProjected(projSet, readerSchema.metadata("b")));
+    assertFalse(isProjected(projSet, readerSchema.metadata("c")));
   }
 
   @Test
@@ -479,7 +473,7 @@ public class TestScanLevelProjection extends SubOperatorTest {
     ProjectionFilter projSet = scanProj.readerProjection();
     assertTrue(projSet.isProjected("b"));
     try {
-      projSet.isProjected(readerSchema.metadata("b"));
+      projSet.projection(readerSchema.metadata("b"));
       fail();
     } catch (UserException e) {
       // Expected
@@ -510,7 +504,7 @@ public class TestScanLevelProjection extends SubOperatorTest {
     ProjectionFilter projSet = scanProj.readerProjection();
     assertTrue(projSet.isProjected("b"));
     try {
-      projSet.isProjected(readerSchema.metadata("b"));
+      isProjected(projSet, readerSchema.metadata("b"));
       fail();
     } catch (UserException e) {
       // Expected
@@ -541,12 +535,9 @@ public class TestScanLevelProjection extends SubOperatorTest {
         .buildSchema();
 
     ProjectionFilter projSet = scanProj.readerProjection();
-    assertTrue(projSet.isProjected("a"));
-    assertTrue(projSet.isProjected(readerSchema.metadata("a")));
-    assertFalse(projSet.isProjected("b"));
-    assertFalse(projSet.isProjected(readerSchema.metadata("b")));
-    assertTrue(projSet.isProjected("c"));
-    assertTrue(projSet.isProjected(readerSchema.metadata("c")));
+    assertTrue(isProjected(projSet, readerSchema.metadata("a")));
+    assertFalse(isProjected(projSet, readerSchema.metadata("b")));
+    assertTrue(isProjected(projSet, readerSchema.metadata("c")));
   }
 
   /**
@@ -575,11 +566,8 @@ public class TestScanLevelProjection extends SubOperatorTest {
         .buildSchema();
 
     ProjectionFilter projSet = scanProj.readerProjection();
-    assertTrue(projSet.isProjected("a"));
-    assertTrue(projSet.isProjected(readerSchema.metadata("a")));
-    assertFalse(projSet.isProjected("b"));
-    assertFalse(projSet.isProjected(readerSchema.metadata("b")));
-    assertFalse(projSet.isProjected("c"));
-    assertFalse(projSet.isProjected(readerSchema.metadata("c")));
+    assertTrue(isProjected(projSet, readerSchema.metadata("a")));
+    assertFalse(isProjected(projSet, readerSchema.metadata("b")));
+    assertFalse(isProjected(projSet, readerSchema.metadata("c")));
   }
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/file/FileScanUtils.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/file/FileScanUtils.java
new file mode 100644
index 0000000..fa5dcd0
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/file/FileScanUtils.java
@@ -0,0 +1,58 @@
+/*
+ * 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.v3.file;
+
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.rowSet.RowSetTestUtils;
+
+public class FileScanUtils {
+
+  // Default file metadata column names; primarily for testing.
+
+  public static final String FILE_NAME_COL = "filename";
+  public static final String FULLY_QUALIFIED_NAME_COL = "fqn";
+  public static final String FILE_PATH_COL = "filepath";
+  public static final String SUFFIX_COL = "suffix";
+  public static final String PARTITION_COL = "dir";
+
+  public static String partitionColName(int partition) {
+    return PARTITION_COL + partition;
+  }
+
+  public static List<SchemaPath> expandMetadata(int dirCount) {
+    List<String> selected = Arrays.asList(
+        FULLY_QUALIFIED_NAME_COL,
+        FILE_PATH_COL,
+        FILE_NAME_COL,
+        SUFFIX_COL);
+
+    for (int i = 0; i < dirCount; i++) {
+      selected.add(PARTITION_COL + Integer.toString(i));
+    }
+    return RowSetTestUtils.projectList(selected);
+  }
+
+  public static List<SchemaPath> projectAllWithMetadata(int dirCount) {
+    return RowSetTestUtils.concat(
+        RowSetTestUtils.projectAll(),
+        expandMetadata(dirCount));
+  }
+}
diff --git a/common/src/main/java/org/apache/drill/common/exceptions/EmptyErrorContext.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/file/MockFileNames.java
similarity index 50%
copy from common/src/main/java/org/apache/drill/common/exceptions/EmptyErrorContext.java
copy to exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/file/MockFileNames.java
index 9589714..ad3dc01 100644
--- a/common/src/main/java/org/apache/drill/common/exceptions/EmptyErrorContext.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/file/MockFileNames.java
@@ -15,12 +15,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.common.exceptions;
+package org.apache.drill.exec.physical.impl.scan.v3.file;
 
-import org.apache.drill.common.exceptions.UserException.Builder;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.ImplicitColumnResolver;
+import org.apache.hadoop.fs.Path;
 
-public class EmptyErrorContext implements CustomErrorContext {
+public interface MockFileNames {
 
-  @Override
-  public void addContext(Builder builder) { }
+  String MOCK_FILE_NAME = "foo.csv";
+  String MOCK_FILE_DIR_PATH = "/w/x/y";
+  String MOCK_FILE_FQN = MOCK_FILE_DIR_PATH + "/" + MOCK_FILE_NAME;
+  String MOCK_FILE_SYSTEM_NAME = "file:" + MOCK_FILE_FQN;
+  Path MOCK_ROOT_PATH = new Path("file:/w");
+  String MOCK_SUFFIX = "csv";
+  String MOCK_DIR0 = "x";
+  String MOCK_DIR1 = "y";
+  Path MOCK_FILE_PATH = new Path(MOCK_FILE_SYSTEM_NAME);
+
+  MajorType IMPLICIT_COL_TYPE = ImplicitColumnResolver.IMPLICIT_COL_TYPE;
+  MajorType PARTITION_COL_TYPE = ImplicitColumnResolver.PARTITION_COL_TYPE;
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/file/TestFileDescrip.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/file/TestFileDescrip.java
new file mode 100644
index 0000000..24563c1
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/file/TestFileDescrip.java
@@ -0,0 +1,149 @@
+/*
+ * 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.v3.file;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.apache.drill.categories.EvfTests;
+import org.apache.drill.test.BaseTest;
+import org.apache.hadoop.fs.Path;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(EvfTests.class)
+public class TestFileDescrip extends BaseTest {
+
+  /**
+   * Degenerate case: no file or root
+   */
+  @Test
+  public void testEmpty() {
+    FileDescrip fd = new FileDescrip(null, null);
+    assertFalse(fd.isSet());
+    assertNull(fd.filePath());
+    assertEquals(0, fd.dirPathLength());
+    assertNull(fd.partition(0));
+  }
+
+  /**
+   * Degenerate case: no file path, but with as selection root
+   * Should never occur in practice.
+   */
+  @Test
+  public void testNoPath() {
+    Path root = new Path("hdfs://a/b");
+    FileDescrip fd = new FileDescrip(null, root);
+    assertFalse(fd.isSet());
+    assertNull(fd.filePath());
+    assertEquals(0, fd.dirPathLength());
+    assertNull(fd.partition(0));
+  }
+
+  /**
+   * Simple file, no selection root.
+   * Should never really occur, but let's test it anyway.
+   */
+  @Test
+  public void testNoRoot() {
+    Path input = new Path("hdfs://foo.csv");
+    FileDescrip fd = new FileDescrip(input, null);
+    assertTrue(fd.isSet());
+    assertSame(input, fd.filePath());
+    assertEquals(0, fd.dirPathLength());
+    assertNull(fd.partition(0));
+  }
+
+  /**
+   * Normal file, no selection root.
+   */
+  @Test
+  public void testSingleFile() {
+    Path input = new Path("hdfs://a/b/c/foo.csv");
+    FileDescrip fd = new FileDescrip(input, null);
+    assertTrue(fd.isSet());
+    assertSame(input, fd.filePath());
+    assertEquals(0, fd.dirPathLength());
+    assertNull(fd.partition(0));
+  }
+
+  /**
+   * Normal file, resides in selection root.
+   */
+  @Test
+  public void testRootFile() {
+    Path root = new Path("hdfs://a/b");
+    Path input = new Path("hdfs://a/b/foo.csv");
+    FileDescrip fd = new FileDescrip(input, root);
+    assertTrue(fd.isSet());
+    assertSame(input, fd.filePath());
+    assertEquals(0, fd.dirPathLength());
+    assertNull(fd.partition(0));
+  }
+
+  /**
+   * Normal file, below selection root.
+   */
+  @Test
+  public void testBelowRoot() {
+    Path root = new Path("hdfs://a/b");
+    Path input = new Path("hdfs://a/b/c/foo.csv");
+    FileDescrip fd = new FileDescrip(input, root);
+    assertTrue(fd.isSet());
+    assertSame(input, fd.filePath());
+    assertEquals(1, fd.dirPathLength());
+    assertEquals("c", fd.partition(0));
+    assertNull(fd.partition(1));
+  }
+
+  /**
+   * Normal file, above selection root.
+   * This is an error condition.
+   */
+  @Test
+  public void testAboveRoot() {
+    Path root = new Path("hdfs://a/b");
+    Path input = new Path("hdfs://a/foo.csv");
+    try {
+      new FileDescrip(input, root);
+      fail();
+    } catch (IllegalArgumentException e) {
+      // Expected
+    }
+  }
+
+  /**
+   * Normal file, disjoint with selection root.
+   * This is an error condition.
+   */
+  @Test
+  public void testDisjointPath() {
+    Path root = new Path("hdfs://a/b");
+    Path input = new Path("hdfs://d/foo.csv");
+    try {
+      new FileDescrip(input, root);
+      fail();
+    } catch (IllegalArgumentException e) {
+      // Expected
+    }
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/file/TestImplicitColumnResolver.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/file/TestImplicitColumnResolver.java
new file mode 100644
index 0000000..a87e330
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/file/TestImplicitColumnResolver.java
@@ -0,0 +1,555 @@
+/*
+ * 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.v3.file;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.drill.categories.EvfTests;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.EmptyErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.ScanTestUtils;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.ImplicitColumnResolver;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.ImplicitColumnResolver.ImplicitColumnOptions;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.ImplicitColumnResolver.ParseResult;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.MutableTupleSchema;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.MutableTupleSchema.ColumnHandle;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.ProjectionSchemaTracker;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.ScanProjectionParser;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.ScanProjectionParser.ProjectionParseResult;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.ScanSchemaConfigBuilder;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.ScanSchemaTracker;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.ScanSchemaTracker.ProjectionType;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.SchemaBasedTracker;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.SchemaUtils;
+import org.apache.drill.exec.physical.rowSet.RowSetTestUtils;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.test.SubOperatorTest;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(EvfTests.class)
+public class TestImplicitColumnResolver extends SubOperatorTest {
+
+  private static final CustomErrorContext ERROR_CONTEXT = EmptyErrorContext.INSTANCE;
+
+  private static class ParserFixture {
+
+    public final ImplicitColumnOptions options;
+    public final ProjectionSchemaTracker tracker;
+
+    public ParserFixture(Collection<SchemaPath> projList) {
+      ProjectionParseResult result = ScanProjectionParser.parse(projList);
+      tracker = new ProjectionSchemaTracker(result, true, EmptyErrorContext.INSTANCE);
+      options = new ImplicitColumnOptions()
+          .optionSet(fixture.getOptionManager());
+   }
+
+    public ParseResult parseImplicit() {
+      ImplicitColumnResolver parser = new ImplicitColumnResolver(options, ERROR_CONTEXT);
+      return parser.parse(tracker);
+    }
+  }
+
+  private boolean isImplicit(List<ColumnHandle> cols, int index) {
+    return SchemaUtils.isImplicit(cols.get(index).column());
+  }
+
+  @Test
+  public void testNoImplicitCols() {
+    ParserFixture parseFixture = new ParserFixture(
+        RowSetTestUtils.projectList("a", "b", "c"));
+    ParseResult result = parseFixture.parseImplicit();
+    assertTrue(result.columns().isEmpty());
+    assertTrue(result.schema().isEmpty());
+  }
+
+  /**
+   * Test including file implicit columns in the project list.
+   */
+  @Test
+  public void testFileImplicitColumnSelection() {
+    // Simulate SELECT a, fqn, filEPath, filename, suffix ...
+    ParserFixture parseFixture = new ParserFixture(
+        RowSetTestUtils.projectList("a",
+            ScanTestUtils.FULLY_QUALIFIED_NAME_COL,
+            "filEPath", // Sic, to test case sensitivity
+            ScanTestUtils.FILE_NAME_COL,
+            ScanTestUtils.SUFFIX_COL));
+    ParseResult result = parseFixture.parseImplicit();
+
+    assertEquals(4, result.columns().size());
+
+    TupleMetadata expected = new SchemaBuilder()
+        .add(ScanTestUtils.FULLY_QUALIFIED_NAME_COL, MinorType.VARCHAR)
+        .add("filEPath", MinorType.VARCHAR)
+        .add(ScanTestUtils.FILE_NAME_COL, MinorType.VARCHAR)
+        .add(ScanTestUtils.SUFFIX_COL, MinorType.VARCHAR)
+        .build();
+    assertEquals(expected, result.schema());
+
+    List<ColumnHandle> cols = parseFixture.tracker.internalSchema().columns();
+    assertFalse(isImplicit(cols, 0));
+    assertTrue(isImplicit(cols, 1));
+    assertTrue(isImplicit(cols, 2));
+    assertTrue(isImplicit(cols, 3));
+    assertTrue(isImplicit(cols, 4));
+  }
+
+  @Test
+  public void testPartitionColumnSelection() {
+
+    String dir0 = ScanTestUtils.partitionColName(0);
+    // Sic: case insensitivity, but name in project list
+    // is preferred over "natural" name.
+    String dir1 = "DIR1";
+    String dir2 = ScanTestUtils.partitionColName(2);
+    ParserFixture parseFixture = new ParserFixture(
+        RowSetTestUtils.projectList(dir2, dir1, dir0, "a"));
+    ParseResult result = parseFixture.parseImplicit();
+    assertEquals(3, result.columns().size());
+
+    TupleMetadata expected = new SchemaBuilder()
+        .addNullable(dir2, MinorType.VARCHAR)
+        .addNullable(dir1, MinorType.VARCHAR)
+        .addNullable(dir0, MinorType.VARCHAR)
+        .build();
+    assertEquals(expected, result.schema());
+
+    List<ColumnHandle> cols = parseFixture.tracker.internalSchema().columns();
+    assertTrue(isImplicit(cols, 0));
+    assertTrue(isImplicit(cols, 1));
+    assertTrue(isImplicit(cols, 2));
+    assertFalse(isImplicit(cols, 3));
+  }
+
+  @Test
+  public void testLegacyWildcard() {
+    ParserFixture parseFixture = new ParserFixture(
+        RowSetTestUtils.projectAll());
+    parseFixture.options
+        .maxPartitionDepth(3)
+        .useLegacyWildcardExpansion(true);
+    ParseResult result = parseFixture.parseImplicit();
+    assertEquals(3, result.columns().size());
+
+    TupleMetadata expected = new SchemaBuilder()
+        .addNullable(ScanTestUtils.partitionColName(0), MinorType.VARCHAR)
+        .addNullable(ScanTestUtils.partitionColName(1), MinorType.VARCHAR)
+        .addNullable(ScanTestUtils.partitionColName(2), MinorType.VARCHAR)
+        .build();
+    assertEquals(expected, result.schema());
+
+    List<ColumnHandle> cols = parseFixture.tracker.internalSchema().columns();
+    assertTrue(isImplicit(cols, 0));
+    assertTrue(isImplicit(cols, 1));
+    assertTrue(isImplicit(cols, 2));
+  }
+
+  @Test
+  public void testRevisedWildcard() {
+    ParserFixture parseFixture = new ParserFixture(
+        RowSetTestUtils.projectAll());
+    parseFixture.options
+        .maxPartitionDepth(3)
+        .useLegacyWildcardExpansion(false);
+    ParseResult result = parseFixture.parseImplicit();
+
+    assertTrue(result.columns().isEmpty());
+    assertTrue(result.schema().isEmpty());
+    assertTrue(parseFixture.tracker.internalSchema().columns().isEmpty());
+  }
+
+  /**
+   * Combine wildcard and file metadata columns. The wildcard expands
+   * table columns but not metadata columns.
+   */
+  @Test
+  public void testLegacyWildcardAndImplictCols() {
+    ParserFixture parseFixture = new ParserFixture(
+        RowSetTestUtils.projectList(
+            SchemaPath.DYNAMIC_STAR,
+            ScanTestUtils.FILE_NAME_COL,
+            ScanTestUtils.SUFFIX_COL));
+    parseFixture.options
+        .maxPartitionDepth(2)
+        .useLegacyWildcardExpansion(true);
+    ParseResult result = parseFixture.parseImplicit();
+
+    TupleMetadata expected = new SchemaBuilder()
+        .addNullable(ScanTestUtils.partitionColName(0), MinorType.VARCHAR)
+        .addNullable(ScanTestUtils.partitionColName(1), MinorType.VARCHAR)
+        .add(ScanTestUtils.FILE_NAME_COL, MinorType.VARCHAR)
+        .add(ScanTestUtils.SUFFIX_COL, MinorType.VARCHAR)
+        .build();
+    assertEquals(expected, result.schema());
+  }
+
+  /**
+   * As above, but include implicit columns before and after the wildcard.
+   * Include both a wildcard and a partition column. The wildcard, in legacy
+   * mode, will create partition columns for any partitions not mentioned in the
+   * project list.
+   */
+  @Test
+  public void testLegacyWildcardAndImplicitColsMixed() {
+    ParserFixture parseFixture = new ParserFixture(
+        RowSetTestUtils.projectList(
+            ScanTestUtils.FILE_NAME_COL,
+            SchemaPath.DYNAMIC_STAR,
+            ScanTestUtils.SUFFIX_COL,
+            ScanTestUtils.partitionColName(0)));
+    parseFixture.options
+        .maxPartitionDepth(3)
+        .useLegacyWildcardExpansion(true);
+    ParseResult result = parseFixture.parseImplicit();
+
+    TupleMetadata expected = new SchemaBuilder()
+        .add(ScanTestUtils.FILE_NAME_COL, MinorType.VARCHAR)
+        .addNullable(ScanTestUtils.partitionColName(1), MinorType.VARCHAR)
+        .addNullable(ScanTestUtils.partitionColName(2), MinorType.VARCHAR)
+        .add(ScanTestUtils.SUFFIX_COL, MinorType.VARCHAR)
+        .addNullable(ScanTestUtils.partitionColName(0), MinorType.VARCHAR)
+        .build();
+    assertEquals(expected, result.schema());
+  }
+
+  /**
+   * Verify that names that look like metadata columns, but appear
+   * to be maps or arrays, are not interpreted as metadata. That is,
+   * the projected table map or array "shadows" the metadata column.
+   */
+  @Test
+  public void testShadowed() {
+    ParserFixture parseFixture = new ParserFixture(
+        RowSetTestUtils.projectList(
+            ScanTestUtils.FILE_NAME_COL + ".a",
+            ScanTestUtils.FILE_PATH_COL + "[0]",
+            ScanTestUtils.partitionColName(0) + ".b",
+            ScanTestUtils.partitionColName(1) + "[0]",
+            ScanTestUtils.SUFFIX_COL));
+    ParseResult result = parseFixture.parseImplicit();
+
+    TupleMetadata expected = new SchemaBuilder()
+        .add(ScanTestUtils.SUFFIX_COL, MinorType.VARCHAR)
+        .build();
+    assertEquals(expected, result.schema());
+
+    List<ColumnHandle> cols = parseFixture.tracker.internalSchema().columns();
+    assertFalse(isImplicit(cols, 0));
+    assertFalse(isImplicit(cols, 1));
+    assertFalse(isImplicit(cols, 2));
+    assertFalse(isImplicit(cols, 3));
+    assertTrue(isImplicit(cols, 4));
+  }
+
+  @Test
+  public void testProvidedImplicitCols() {
+
+    TupleMetadata providedSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .add("myFqn", MinorType.VARCHAR)
+        // Sic, to test case nullable
+        .addNullable("myFilePath", MinorType.VARCHAR)
+        .add("myFileName", MinorType.VARCHAR)
+        .add("mySuffix", MinorType.VARCHAR)
+        .addNullable("myDir", MinorType.VARCHAR)
+        .build();
+    SchemaUtils.markImplicit(providedSchema.metadata("myFqn"), ColumnMetadata.IMPLICIT_FQN);
+    // Sic, to test case sensitivity
+    SchemaUtils.markImplicit(providedSchema.metadata("myFilePath"), ColumnMetadata.IMPLICIT_FILEPATH.toUpperCase());
+    SchemaUtils.markImplicit(providedSchema.metadata("myFileName"), ColumnMetadata.IMPLICIT_FILENAME);
+    SchemaUtils.markImplicit(providedSchema.metadata("mySuffix"), ColumnMetadata.IMPLICIT_SUFFIX);
+    SchemaUtils.markAsPartition(providedSchema.metadata("myDir"), 0);
+
+    ParserFixture parseFixture = new ParserFixture(
+        RowSetTestUtils.projectAll());
+    parseFixture.tracker.applyProvidedSchema(providedSchema);
+    parseFixture.options
+        .maxPartitionDepth(1);
+    ParseResult result = parseFixture.parseImplicit();
+
+    assertEquals(5, result.columns().size());
+
+    TupleMetadata expected = new SchemaBuilder()
+        .add("myFqn", MinorType.VARCHAR)
+        .addNullable("myFilePath", MinorType.VARCHAR)
+        .add("myFileName", MinorType.VARCHAR)
+        .add("mySuffix", MinorType.VARCHAR)
+        .addNullable("myDir", MinorType.VARCHAR)
+        .build();
+    assertEquals(expected, result.schema());
+
+    List<ColumnHandle> cols = parseFixture.tracker.internalSchema().columns();
+    assertFalse(isImplicit(cols, 0));
+    assertTrue(isImplicit(cols, 1));
+    assertTrue(isImplicit(cols, 2));
+    assertTrue(isImplicit(cols, 3));
+    assertTrue(isImplicit(cols, 4));
+    assertTrue(isImplicit(cols, 5));
+  }
+
+
+  @Test
+  public void testProvidedImplicitMatchesProject() {
+
+    TupleMetadata providedSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .add("fqn", MinorType.VARCHAR)
+        // Sic, to test case sensitivity
+        .add("filePath", MinorType.VARCHAR)
+        .addNullable("dir0", MinorType.VARCHAR)
+        .build();
+    SchemaUtils.markImplicit(providedSchema.metadata("fqn"), ColumnMetadata.IMPLICIT_FQN);
+    SchemaUtils.markImplicit(providedSchema.metadata("filePath"), ColumnMetadata.IMPLICIT_FILEPATH.toUpperCase());
+    SchemaUtils.markAsPartition(providedSchema.metadata("dir0"), 0);
+
+    ParserFixture parseFixture = new ParserFixture(
+        RowSetTestUtils.projectAll());
+    parseFixture.tracker.applyProvidedSchema(providedSchema);
+    parseFixture.options.maxPartitionDepth(1);
+    ParseResult result = parseFixture.parseImplicit();
+
+    assertEquals(3, result.columns().size());
+
+    TupleMetadata expected = new SchemaBuilder()
+        .add("fqn", MinorType.VARCHAR)
+        .add("filePath", MinorType.VARCHAR)
+        .addNullable("dir0", MinorType.VARCHAR)
+        .build();
+    assertEquals(expected, result.schema());
+
+    List<ColumnHandle> cols = parseFixture.tracker.internalSchema().columns();
+    assertFalse(isImplicit(cols, 0));
+    assertTrue(isImplicit(cols, 1));
+    assertTrue(isImplicit(cols, 2));
+    assertTrue(isImplicit(cols, 3));
+  }
+
+  @Test
+  public void testProvidedImplicitColTypeConflict() {
+    TupleMetadata providedSchema = new SchemaBuilder()
+        .add("myFqn", MinorType.INT)
+        .build();
+    SchemaUtils.markImplicit(providedSchema.metadata("myFqn"), ColumnMetadata.IMPLICIT_FQN);
+
+    ParserFixture parseFixture = new ParserFixture(
+        RowSetTestUtils.projectAll());
+    parseFixture.tracker.applyProvidedSchema(providedSchema);
+    try {
+      parseFixture.parseImplicit();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains("wrong type"));
+    }
+  }
+
+  @Test
+  public void testProvidedImplicitColModeConflict() {
+    TupleMetadata providedSchema = new SchemaBuilder()
+        .addArray("myFqn", MinorType.VARCHAR)
+        .build();
+    SchemaUtils.markImplicit(providedSchema.metadata("myFqn"), ColumnMetadata.IMPLICIT_FQN);
+
+    ParserFixture parseFixture = new ParserFixture(
+        RowSetTestUtils.projectAll());
+    parseFixture.tracker.applyProvidedSchema(providedSchema);
+    try {
+      parseFixture.parseImplicit();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains("wrong type"));
+    }
+  }
+
+  @Test
+  public void testProvidedPartitionColTypeConflict() {
+    TupleMetadata providedSchema = new SchemaBuilder()
+        .addNullable("myDir", MinorType.INT)
+        .build();
+    SchemaUtils.markAsPartition(providedSchema.metadata("myDir"), 0);
+
+    ParserFixture parseFixture = new ParserFixture(
+        RowSetTestUtils.projectAll());
+    parseFixture.tracker.applyProvidedSchema(providedSchema);
+    parseFixture.options.maxPartitionDepth(1);
+    try {
+      parseFixture.parseImplicit();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains("wrong type"));
+    }
+  }
+
+  @Test
+  public void testProvidedPartitionColModeConflict() {
+    TupleMetadata providedSchema = new SchemaBuilder()
+        .add("myDir", MinorType.VARCHAR)
+        .build();
+    SchemaUtils.markAsPartition(providedSchema.metadata("myDir"), 0);
+
+    ParserFixture parseFixture = new ParserFixture(
+        RowSetTestUtils.projectAll());
+    parseFixture.tracker.applyProvidedSchema(providedSchema);
+    parseFixture.options.maxPartitionDepth(1);
+    try {
+      parseFixture.parseImplicit();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains("wrong type"));
+    }
+  }
+
+  @Test
+  public void testProvidedUndefinedImplicitCol() {
+    TupleMetadata providedSchema = new SchemaBuilder()
+        .add("myDir", MinorType.VARCHAR)
+        .build();
+    SchemaUtils.markImplicit(providedSchema.metadata("myDir"), "bogus");
+
+    ParserFixture parseFixture = new ParserFixture(
+        RowSetTestUtils.projectAll());
+    parseFixture.tracker.applyProvidedSchema(providedSchema);
+    parseFixture.options.maxPartitionDepth(1);
+    try {
+      parseFixture.parseImplicit();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains("references an undefined implicit column"));
+    }
+  }
+
+  @Test
+  public void testImplicitOnly() {
+    ParserFixture parseFixture = new ParserFixture(
+        RowSetTestUtils.projectList(
+            ScanTestUtils.FULLY_QUALIFIED_NAME_COL,
+            ScanTestUtils.FILE_NAME_COL));
+    ParseResult result = parseFixture.parseImplicit();
+
+    assertEquals(2, result.columns().size());
+    assertTrue(parseFixture.tracker.isResolved());
+    assertSame(ProjectionType.NONE, parseFixture.tracker.projectionType());
+  }
+
+  @Test
+  public void testImplicitOnlyWildcard() {
+    ParserFixture parseFixture = new ParserFixture(
+        RowSetTestUtils.projectList(
+            ScanTestUtils.FULLY_QUALIFIED_NAME_COL,
+            SchemaPath.DYNAMIC_STAR,
+            ScanTestUtils.FILE_NAME_COL));
+    ParseResult result = parseFixture.parseImplicit();
+
+    assertEquals(2, result.columns().size());
+    assertTrue(parseFixture.tracker.isResolved());
+    assertSame(ProjectionType.ALL, parseFixture.tracker.projectionType());
+  }
+
+  /**
+   * The scan framework should expand partitions after table columns.
+   */
+  @Test
+  public void testPartitionExpansionPlacement() {
+    // Parse out implicit columns at start of scan
+    ParserFixture parseFixture = new ParserFixture(
+        RowSetTestUtils.projectAll());
+    parseFixture.options
+        .maxPartitionDepth(2)
+        .useLegacyWildcardExpansion(true);
+    ParseResult result = parseFixture.parseImplicit();
+
+    // Later resolve the table schema
+    TupleMetadata tableSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .add("f", MinorType.BIGINT)
+        .build();
+    parseFixture.tracker.applyReaderSchema(tableSchema, ERROR_CONTEXT);
+
+    TupleMetadata expected = new SchemaBuilder()
+        .addNullable(ScanTestUtils.partitionColName(0), MinorType.VARCHAR)
+        .addNullable(ScanTestUtils.partitionColName(1), MinorType.VARCHAR)
+        .build();
+    assertEquals(expected, result.schema());
+
+    // Implicit columns follow the table columns
+    expected = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .add("f", MinorType.BIGINT)
+        .addNullable(ScanTestUtils.partitionColName(0), MinorType.VARCHAR)
+        .addNullable(ScanTestUtils.partitionColName(1), MinorType.VARCHAR)
+        .build();
+    assertEquals(expected, parseFixture.tracker.outputSchema());
+  }
+
+  @Test
+  public void testImplicitWithDefinedSchema() {
+    final ScanSchemaConfigBuilder builder = new ScanSchemaConfigBuilder()
+        .projection(RowSetTestUtils.projectList(
+            "a", "b", "c",
+            ScanTestUtils.FILE_NAME_COL,
+            ScanTestUtils.FILE_PATH_COL,
+            ScanTestUtils.partitionColName(0),
+            ScanTestUtils.partitionColName(2)));
+
+    final TupleMetadata definedSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .add("b", MinorType.BIGINT)
+        .add("c", MinorType.VARCHAR)
+        .add(ScanTestUtils.FILE_NAME_COL, MinorType.VARCHAR)
+        .add(ScanTestUtils.FILE_PATH_COL, MinorType.VARCHAR)
+        .addNullable(ScanTestUtils.partitionColName(0), MinorType.VARCHAR)
+        .addNullable(ScanTestUtils.partitionColName(2), MinorType.VARCHAR)
+        .buildSchema();
+
+    // With a defined schema, we have to explicitly mark implicit columns
+    // so the schema is independent of system/session options.
+    SchemaUtils.markImplicit(definedSchema.metadata(ScanTestUtils.FILE_NAME_COL), ColumnMetadata.IMPLICIT_FILENAME);
+    SchemaUtils.markImplicit(definedSchema.metadata(ScanTestUtils.FILE_PATH_COL), ScanTestUtils.FILE_PATH_COL);
+    SchemaUtils.markAsPartition(definedSchema.metadata(ScanTestUtils.partitionColName(0)), 0);
+    SchemaUtils.markAsPartition(definedSchema.metadata(ScanTestUtils.partitionColName(2)), 2);
+    builder.definedSchema(definedSchema);
+
+    final ScanSchemaTracker schemaTracker = builder.build();
+    assertTrue(schemaTracker instanceof SchemaBasedTracker);
+    assertTrue(schemaTracker.isResolved());
+    assertSame(ProjectionType.SOME, schemaTracker.projectionType());
+
+    // Implicit columns should be marked
+    MutableTupleSchema internalSchema = schemaTracker.internalSchema();
+    assertFalse(internalSchema.find("a").isImplicit());
+    assertTrue(internalSchema.find(ScanTestUtils.FILE_NAME_COL).isImplicit());
+    assertTrue(internalSchema.find(ScanTestUtils.partitionColName(2)).isImplicit());
+
+    ImplicitColumnOptions options = new ImplicitColumnOptions()
+        .optionSet(fixture.getOptionManager());
+    ImplicitColumnResolver parser = new ImplicitColumnResolver(options, ERROR_CONTEXT);
+    ParseResult result = parser.parse(schemaTracker);
+    assertEquals(4, result.columns().size());
+
+    TupleMetadata readerInputSchema = schemaTracker.readerInputSchema();
+    assertEquals(3, readerInputSchema.size());
+
+    assertEquals(definedSchema, schemaTracker.outputSchema());
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/schema/BaseTestSchemaTracker.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/schema/BaseTestSchemaTracker.java
new file mode 100644
index 0000000..c047c24
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/schema/BaseTestSchemaTracker.java
@@ -0,0 +1,71 @@
+/*
+ * 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.v3.schema;
+
+import java.util.Collection;
+
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.EmptyErrorContext;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.ScanProjectionParser.ProjectionParseResult;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.test.BaseTest;
+
+public class BaseTestSchemaTracker extends BaseTest {
+
+  protected static final CustomErrorContext ERROR_CONTEXT = EmptyErrorContext.INSTANCE;
+  protected static final String MOCK_PROP = "my.prop";
+  protected static final String MOCK_VALUE = "my-value";
+
+  protected static final TupleMetadata SCHEMA = new SchemaBuilder()
+      .add("a", MinorType.INT)
+      .addNullable("b", MinorType.VARCHAR)
+      .build();
+
+  protected static final TupleMetadata MAP_SCHEMA = new SchemaBuilder()
+      .add("a", MinorType.INT)
+      .addMap("m")
+        .add("x", MinorType.BIGINT)
+        .add("y", MinorType.VARCHAR)
+        .resumeSchema()
+      .buildSchema();
+
+  protected static final TupleMetadata NESTED_MAP_SCHEMA = new SchemaBuilder()
+      .add("a", MinorType.INT)
+      .addMap("m")
+        .add("x", MinorType.BIGINT)
+        .add("y", MinorType.VARCHAR)
+        .addMap("m2")
+          .add("p", MinorType.BIGINT)
+          .add("q", MinorType.VARCHAR)
+          .resumeMap()
+        .resumeSchema()
+      .buildSchema();
+
+  static {
+    SCHEMA.metadata("a").setProperty(MOCK_PROP, MOCK_VALUE);
+    MAP_SCHEMA.metadata("m").setProperty(MOCK_PROP, MOCK_VALUE);
+  }
+
+  protected static ProjectionSchemaTracker trackerFor(Collection<SchemaPath> projList) {
+    ProjectionParseResult result = ScanProjectionParser.parse(projList);
+    return new ProjectionSchemaTracker(result, true, ERROR_CONTEXT);
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/schema/TestDynamicSchemaFilter.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/schema/TestDynamicSchemaFilter.java
new file mode 100644
index 0000000..4dcc4ec
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/schema/TestDynamicSchemaFilter.java
@@ -0,0 +1,179 @@
+/*
+ * 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.v3.schema;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.drill.categories.EvfTests;
+import org.apache.drill.common.exceptions.EmptyErrorContext;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.DynamicSchemaFilter.DynamicTupleFilter;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.ScanProjectionParser.ProjectionParseResult;
+import org.apache.drill.exec.physical.resultSet.impl.ProjectionFilter;
+import org.apache.drill.exec.physical.resultSet.impl.ProjectionFilter.ProjResult;
+import org.apache.drill.exec.physical.rowSet.RowSetTestUtils;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(EvfTests.class)
+public class TestDynamicSchemaFilter {
+
+  private static final ColumnMetadata A_COL =
+      MetadataUtils.newScalar("a", Types.required(MinorType.INT));
+  private static final ColumnMetadata B_COL =
+      MetadataUtils.newScalar("b", Types.optional(MinorType.VARCHAR));
+  private static final ColumnMetadata SPECIAL_COLUMN =
+      MetadataUtils.newScalar("special", Types.required(MinorType.INT));
+  private static final ColumnMetadata SPECIAL_COLUMN2 =
+      MetadataUtils.newScalar("special2", Types.required(MinorType.INT));
+  private static final ColumnMetadata MAP_COL = new SchemaBuilder()
+        .addMap("m")
+          .add("x", MinorType.BIGINT)
+          .add("y", MinorType.VARCHAR)
+          .resumeSchema()
+        .build()
+        .metadata("m");
+
+  static {
+    SchemaUtils.markExcludeFromWildcard(SPECIAL_COLUMN);
+    SchemaUtils.markExcludeFromWildcard(SPECIAL_COLUMN2);
+  }
+
+  @Test
+  public void testProjectAll() {
+    ProjectionFilter filter =ProjectionFilter.PROJECT_ALL;
+    assertFalse(filter.isEmpty());
+
+    assertTrue(filter.isProjected(A_COL.name()));
+    ProjResult result = filter.projection(A_COL);
+    assertTrue(result.isProjected);
+    assertNull(result.projection);
+
+    assertTrue(filter.isProjected(MAP_COL.name()));
+    result = filter.projection(MAP_COL);
+    assertTrue(result.isProjected);
+    assertNull(result.projection);
+    assertSame(ProjectionFilter.PROJECT_ALL, result.mapFilter);
+
+    // "Special" columns are projected only by name, but rely on
+    // a property in the column itself, so give inconsistent answers
+    // from the filter.
+    assertTrue(filter.isProjected(SPECIAL_COLUMN.name()));
+    result = filter.projection(SPECIAL_COLUMN);
+    assertFalse(result.isProjected);
+  }
+
+  @Test
+  public void testEmptyProjectList() {
+    ProjectionFilter filter = ProjectionFilter.PROJECT_NONE;
+    assertTrue(filter.isEmpty());
+
+    assertFalse(filter.isProjected(A_COL.name()));
+    ProjResult result = filter.projection(A_COL);
+    assertFalse(result.isProjected);
+
+    assertFalse(filter.isProjected(MAP_COL.name()));
+    result = filter.projection(MAP_COL);
+    assertFalse(result.isProjected);
+    assertSame(ProjectionFilter.PROJECT_NONE, result.mapFilter);
+
+    assertFalse(filter.isProjected(SPECIAL_COLUMN.name()));
+    result = filter.projection(SPECIAL_COLUMN);
+    assertFalse(result.isProjected);
+  }
+
+  @Test
+  public void testProjectList() {
+    ProjectionParseResult parseResult = ScanProjectionParser.parse(
+        RowSetTestUtils.projectList("a", "c", "m", "special"));
+    ProjectionFilter filter = new DynamicTupleFilter(
+        parseResult.dynamicSchema, EmptyErrorContext.INSTANCE);
+    assertFalse(filter.isEmpty());
+
+    assertTrue(filter.isProjected(A_COL.name()));
+    ProjResult result = filter.projection(A_COL);
+    assertTrue(result.isProjected);
+    assertNotNull(result.projection);
+    assertTrue(result.projection.isDynamic());
+    assertEquals(A_COL.name(), result.projection.name());
+
+    assertFalse(filter.isProjected(B_COL.name()));
+    result = filter.projection(B_COL);
+    assertFalse(result.isProjected);
+
+    assertTrue(filter.isProjected(MAP_COL.name()));
+    result = filter.projection(MAP_COL);
+    assertTrue(result.isProjected);
+    assertNotNull(result.projection);
+    assertTrue(result.projection.isDynamic());
+    assertEquals(MAP_COL.name(), result.projection.name());
+    assertSame(ProjectionFilter.PROJECT_ALL, result.mapFilter);
+
+    assertTrue(filter.isProjected(SPECIAL_COLUMN.name()));
+    result = filter.projection(SPECIAL_COLUMN);
+    assertTrue(result.isProjected);
+    assertNotNull(result.projection);
+    assertTrue(result.projection.isDynamic());
+    assertEquals(SPECIAL_COLUMN.name(), result.projection.name());
+
+    assertFalse(filter.isProjected(SPECIAL_COLUMN2.name()));
+    result = filter.projection(SPECIAL_COLUMN2);
+    assertFalse(result.isProjected);
+  }
+
+  @Test
+  public void testMapProjectList() {
+    ProjectionParseResult parseResult = ScanProjectionParser.parse(
+        RowSetTestUtils.projectList("m.x"));
+    ProjectionFilter filter = new DynamicTupleFilter(
+        parseResult.dynamicSchema, EmptyErrorContext.INSTANCE);
+    assertFalse(filter.isEmpty());
+
+    assertTrue(filter.isProjected(MAP_COL.name()));
+    ProjResult result = filter.projection(MAP_COL);
+    assertTrue(result.isProjected);
+    assertNotNull(result.projection);
+    assertTrue(result.projection.isDynamic());
+    assertEquals(MAP_COL.name(), result.projection.name());
+    assertTrue(result.mapFilter instanceof DynamicTupleFilter);
+
+    ProjectionFilter mapFilter = result.mapFilter;
+
+    ColumnMetadata x_col = MAP_COL.tupleSchema().metadata("x");
+    assertTrue(mapFilter.isProjected("x"));
+    result = mapFilter.projection(x_col);
+    assertTrue(result.isProjected);
+    assertNotNull(result.projection);
+    assertTrue(result.projection.isDynamic());
+    assertEquals(x_col.name(), result.projection.name());
+
+    ColumnMetadata y_col = MAP_COL.tupleSchema().metadata("y");
+    assertFalse(mapFilter.isProjected("y"));
+    result = mapFilter.projection(y_col);
+    assertFalse(result.isProjected);
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/schema/TestProjectedPath.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/schema/TestProjectedPath.java
new file mode 100644
index 0000000..491be8b
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/schema/TestProjectedPath.java
@@ -0,0 +1,357 @@
+/*
+ * 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.v3.schema;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.drill.categories.EvfTests;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.ScanProjectionParser.ProjectionParseResult;
+import org.apache.drill.exec.physical.rowSet.RowSetTestUtils;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+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;
+
+/**
+ * Projection creates a pattern which we match against a particular type
+ * to see if the projection path is consistent with the type. Tests here
+ * verify the consistency checks.
+ */
+@Category(EvfTests.class)
+public class TestProjectedPath {
+
+  // INT is a proxy for all scalar columns.
+  private static final ColumnMetadata INT_COLUMN = intSchema().metadata("a");
+  private static final ColumnMetadata INT_ARRAY_COLUMN = intArraySchema().metadata("a");
+  private static final ColumnMetadata MAP_COLUMN = mapSchema().metadata("a");
+  private static final ColumnMetadata MAP_ARRAY_COLUMN = mapArraySchema().metadata("a");
+  private static final ColumnMetadata UNION_COLUMN = unionSchema().metadata("a");
+  private static final ColumnMetadata LIST_COLUMN = listSchema().metadata("a");
+  private static final ColumnMetadata DICT_INT_INT_COLUMN = dictSchema(MinorType.INT).metadata("a");
+  private static final ColumnMetadata DICT_ARRAY_INT_INT_COLUMN = dictArraySchema(MinorType.INT).metadata("a");
+
+  // Commented out for now as the projected path does not (yet) have sufficient
+  // context to verify projection against a DICT. For other types, the rules are
+  // based purely on the type. (a[0] is for arrays, a.b is for maps.) But, for
+  // DICT, we must consider the type of the keys. So, (dict[0]) is valid if
+  // either the DICT is repeated, or it has an integer key. This logic was attempted,
+  // but turned out to be too complex and brittle, so was backed out for now. The tests
+  // are left here as a record of what we should eventually do.
+//  private static final ColumnMetadata DICT_BIGINT_INT_COLUMN = dictSchema(MinorType.BIGINT).metadata("a");
+//  private static final ColumnMetadata DICT_ARRAY_BIGINT_INT_COLUMN = dictArraySchema(MinorType.BIGINT).metadata("a");
+//  private static final ColumnMetadata DICT_VARCHAR_INT_COLUMN = dictSchema(MinorType.VARCHAR).metadata("a");
+//  private static final ColumnMetadata DICT_ARRAY_VARCHAR_INT_COLUMN = dictArraySchema(MinorType.VARCHAR).metadata("a");
+//  private static final ColumnMetadata DICT_DOUBLE_INT_COLUMN = dictSchema(MinorType.FLOAT8).metadata("a");
+//  private static final ColumnMetadata DICT_ARRAY_DOUBLE_INT_COLUMN = dictArraySchema(MinorType.FLOAT8).metadata("a");
+//  private static final ColumnMetadata DICT_ARRAY_INT_INT_ARRAY_COLUMN = dictArrayArraySchema(MinorType.INT).metadata("a");
+//  private static final ColumnMetadata DICT_ARRAY_VARCHAR_INT_ARRAY_COLUMN = dictArrayArraySchema(MinorType.VARCHAR).metadata("a");
+
+  private static TupleMetadata intSchema() {
+    return new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .build();
+  }
+
+  private static TupleMetadata intArraySchema() {
+    return new SchemaBuilder()
+        .addArray("a", MinorType.INT)
+        .build();
+  }
+
+  private static TupleMetadata mapSchema() {
+    return new SchemaBuilder()
+        .addMap("a")
+          .add("i", MinorType.INT)
+          .addMap("m")
+            .add("mi", MinorType.INT)
+            .resumeMap()
+          .resumeSchema()
+        .build();
+  }
+
+  private static TupleMetadata mapArraySchema() {
+    return new SchemaBuilder()
+        .addMapArray("a")
+          .add("i", MinorType.INT)
+          .addMap("m")
+            .add("mi", MinorType.INT)
+            .resumeMap()
+          .resumeSchema()
+        .build();
+  }
+
+  private static TupleMetadata dictSchema(MinorType keyType) {
+    return new SchemaBuilder()
+        .addDict("a", keyType)
+          .value(MinorType.INT)
+          .resumeSchema()
+        .build();
+  }
+
+  private static TupleMetadata dictArraySchema(MinorType keyType) {
+    return new SchemaBuilder()
+        .addDictArray("a", keyType)
+          .value(MinorType.INT)
+          .resumeSchema()
+        .build();
+  }
+
+  @SuppressWarnings("unused")
+  private static TupleMetadata dictArrayArraySchema(MinorType keyType) {
+    return new SchemaBuilder()
+        .addDictArray("a", keyType)
+          .value(Types.repeated(MinorType.INT))
+          .resumeSchema()
+        .build();
+  }
+
+  private static TupleMetadata unionSchema() {
+    return new SchemaBuilder()
+        .addUnion("a")
+          .addType(MinorType.INT)
+          .resumeSchema()
+        .build();
+  }
+
+  private static TupleMetadata listSchema() {
+    return new SchemaBuilder()
+        .addList("a")
+          .addType(MinorType.INT)
+          .resumeSchema()
+        .build();
+  }
+
+  private void assertConsistent(TupleMetadata projSet, ColumnMetadata col) {
+    assertTrue(isConsistent(projSet, col));
+  }
+
+  private void assertNotConsistent(TupleMetadata projSet, ColumnMetadata col) {
+    assertFalse(isConsistent(projSet, col));
+  }
+
+  private boolean isConsistent(TupleMetadata tuple, ColumnMetadata readCol) {
+    if (tuple == null || SchemaUtils.isProjectAll(tuple)) {
+      return true;
+    }
+    // If the column is projected, it may be projected implicitly.
+    // Only check explicit projection.
+    ProjectedColumn col = (ProjectedColumn) tuple.metadata(readCol.name());
+    if (col == null) {
+      return true;
+    } else {
+      return SchemaUtils.isConsistent(col, readCol);
+    }
+  }
+
+  private void assertAllConsistent(TupleMetadata projSet) {
+    assertConsistent(projSet, INT_COLUMN);
+    assertConsistent(projSet, INT_ARRAY_COLUMN);
+    assertConsistent(projSet, MAP_COLUMN);
+    assertConsistent(projSet, MAP_ARRAY_COLUMN);
+    assertConsistent(projSet, DICT_INT_INT_COLUMN);
+    assertConsistent(projSet, DICT_ARRAY_INT_INT_COLUMN);
+    assertConsistent(projSet, UNION_COLUMN);
+    assertConsistent(projSet, LIST_COLUMN);
+  }
+
+  @Test
+  public void testSimplePath() {
+    ProjectionParseResult result = ScanProjectionParser.parse(
+        RowSetTestUtils.projectList("a"));
+    TupleMetadata projSet = result.dynamicSchema;
+
+    assertAllConsistent(projSet);
+
+    // No constraints on an unprojected column.
+
+    assertTrue(isConsistent(projSet,
+        MetadataUtils.newScalar("b", Types.required(MinorType.INT))));
+  }
+
+  @Test
+  public void testProjectAll() {
+    ProjectionParseResult result = ScanProjectionParser.parse(
+        RowSetTestUtils.projectAll());
+
+    // No constraints on wildcard projection
+    assertAllConsistent(result.dynamicSchema);
+  }
+
+  @Test
+  public void testProjectNone() {
+    ProjectionParseResult result = ScanProjectionParser.parse(
+        RowSetTestUtils.projectNone());
+
+    // No constraints on empty projection
+    assertAllConsistent(result.dynamicSchema);
+  }
+
+  @Test
+  public void test1DArray() {
+    ProjectionParseResult result = ScanProjectionParser.parse(
+        RowSetTestUtils.projectList("a[0]"));
+    TupleMetadata projSet = result.dynamicSchema;
+
+    assertNotConsistent(projSet, INT_COLUMN);
+    assertConsistent(projSet, INT_ARRAY_COLUMN);
+    assertNotConsistent(projSet, MAP_COLUMN);
+    assertConsistent(projSet, MAP_ARRAY_COLUMN);
+    assertConsistent(projSet, UNION_COLUMN);
+    assertConsistent(projSet, LIST_COLUMN);
+
+    assertConsistent(projSet, DICT_INT_INT_COLUMN);
+
+    // TODO: Enforce specific DICT keys, if needed.
+//    assertDictConsistent(projSet, DICT_INT_INT_COLUMN);
+//    assertDictConsistent(projSet, DICT_ARRAY_INT_INT_COLUMN);
+//    assertDictConsistent(projSet, DICT_ARRAY_INT_INT_ARRAY_COLUMN);
+//    assertDictConsistent(projSet, DICT_BIGINT_INT_COLUMN);
+//    assertDictConsistent(projSet, DICT_ARRAY_BIGINT_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_VARCHAR_INT_COLUMN);
+//    assertDictConsistent(projSet, DICT_ARRAY_VARCHAR_INT_COLUMN);
+//    assertDictConsistent(projSet, DICT_ARRAY_VARCHAR_INT_ARRAY_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_DOUBLE_INT_COLUMN);
+//    assertDictConsistent(projSet, DICT_ARRAY_DOUBLE_INT_COLUMN);
+  }
+
+  @Test
+  public void test2DArray() {
+    ProjectionParseResult result = ScanProjectionParser.parse(
+        RowSetTestUtils.projectList("a[0][1]"));
+    TupleMetadata projSet = result.dynamicSchema;
+
+    assertNotConsistent(projSet, INT_COLUMN);
+    assertNotConsistent(projSet, INT_ARRAY_COLUMN);
+    assertNotConsistent(projSet, MAP_COLUMN);
+    assertNotConsistent(projSet, MAP_ARRAY_COLUMN);
+    assertConsistent(projSet, UNION_COLUMN);
+    assertConsistent(projSet, LIST_COLUMN);
+
+    assertConsistent(projSet, DICT_INT_INT_COLUMN);
+
+    // TODO: Enforce specific DICT keys, if needed.
+//    assertDictNotConsistent(projSet, DICT_INT_INT_COLUMN);
+//    assertDictConsistent(projSet, DICT_ARRAY_INT_INT_COLUMN);
+//    assertDictConsistent(projSet, DICT_ARRAY_INT_INT_ARRAY_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_BIGINT_INT_COLUMN);
+//    assertDictConsistent(projSet, DICT_ARRAY_BIGINT_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_VARCHAR_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_ARRAY_VARCHAR_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_ARRAY_VARCHAR_INT_ARRAY_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_DOUBLE_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_ARRAY_DOUBLE_INT_COLUMN);
+  }
+
+  @Test
+  public void test3DArray() {
+    ProjectionParseResult result = ScanProjectionParser.parse(
+        RowSetTestUtils.projectList("a[0][1][2]"));
+    TupleMetadata projSet = result.dynamicSchema;
+
+    assertNotConsistent(projSet, INT_COLUMN);
+    assertNotConsistent(projSet, INT_ARRAY_COLUMN);
+    assertNotConsistent(projSet, MAP_COLUMN);
+    assertNotConsistent(projSet, MAP_ARRAY_COLUMN);
+    assertConsistent(projSet, UNION_COLUMN);
+    assertConsistent(projSet, LIST_COLUMN);
+
+    assertConsistent(projSet, DICT_INT_INT_COLUMN);
+
+    // TODO: Enforce specific DICT keys, if needed.
+//    assertDictNotConsistent(projSet, DICT_INT_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_ARRAY_INT_INT_COLUMN);
+//    assertDictConsistent(projSet, DICT_ARRAY_INT_INT_ARRAY_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_BIGINT_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_ARRAY_BIGINT_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_VARCHAR_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_ARRAY_VARCHAR_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_ARRAY_VARCHAR_INT_ARRAY_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_DOUBLE_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_ARRAY_DOUBLE_INT_COLUMN);
+  }
+
+  @Test
+  public void testMap() {
+    ProjectionParseResult result = ScanProjectionParser.parse(
+        RowSetTestUtils.projectList("a.b"));
+    TupleMetadata projSet = result.dynamicSchema;
+
+    assertNotConsistent(projSet, INT_COLUMN);
+    assertNotConsistent(projSet, INT_ARRAY_COLUMN);
+    assertConsistent(projSet, MAP_COLUMN);
+    assertConsistent(projSet, MAP_ARRAY_COLUMN);
+
+    // A UNION could contain a map, which would allow the
+    // a.b path to be valid.
+    assertConsistent(projSet, UNION_COLUMN);
+    // A LIST could be a list of MAPs, so a.b could mean
+    // to pick out the b column in all array entries.
+    assertConsistent(projSet, LIST_COLUMN);
+
+    assertConsistent(projSet, DICT_INT_INT_COLUMN);
+
+    // TODO: Enforce specific DICT keys, if needed.
+//    assertDictNotConsistent(projSet, DICT_INT_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_ARRAY_INT_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_ARRAY_INT_INT_ARRAY_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_BIGINT_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_ARRAY_BIGINT_INT_COLUMN);
+//    assertDictConsistent(projSet, DICT_VARCHAR_INT_COLUMN);
+//    assertDictConsistent(projSet, DICT_ARRAY_VARCHAR_INT_COLUMN);
+//    assertDictConsistent(projSet, DICT_ARRAY_VARCHAR_INT_ARRAY_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_DOUBLE_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_ARRAY_DOUBLE_INT_COLUMN);
+  }
+
+  @Test
+  public void testMapArray() {
+    ProjectionParseResult result = ScanProjectionParser.parse(
+        RowSetTestUtils.projectList("a[0].b"));
+    TupleMetadata projSet = result.dynamicSchema;
+
+    assertNotConsistent(projSet, INT_COLUMN);
+    assertNotConsistent(projSet, INT_ARRAY_COLUMN);
+    assertNotConsistent(projSet, MAP_COLUMN);
+    assertConsistent(projSet, MAP_ARRAY_COLUMN);
+
+    // A UNION could contain a repeated map, which would allow the
+    // a.b path to be valid.
+    assertConsistent(projSet, UNION_COLUMN);
+    // A LIST could contain MAPs.
+    assertConsistent(projSet, LIST_COLUMN);
+
+    assertConsistent(projSet, DICT_INT_INT_COLUMN);
+
+    // TODO: Enforce specific DICT keys, if needed.
+//    assertDictNotConsistent(projSet, DICT_INT_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_ARRAY_INT_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_ARRAY_INT_INT_ARRAY_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_BIGINT_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_ARRAY_BIGINT_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_VARCHAR_INT_COLUMN);
+//    assertDictConsistent(projSet, DICT_ARRAY_VARCHAR_INT_COLUMN);
+//    assertDictConsistent(projSet, DICT_ARRAY_VARCHAR_INT_ARRAY_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_DOUBLE_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_ARRAY_DOUBLE_INT_COLUMN);
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/schema/TestProjectionParser.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/schema/TestProjectionParser.java
new file mode 100644
index 0000000..02d05a1
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/schema/TestProjectionParser.java
@@ -0,0 +1,472 @@
+/*
+ * 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.v3.schema;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.drill.categories.EvfTests;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.ScanProjectionParser.ProjectionParseResult;
+import org.apache.drill.exec.physical.rowSet.RowSetTestUtils;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.test.BaseTest;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test the projection list parser: parses a list of SchemaPath
+ * items into a detailed structure, handling duplicate or overlapping
+ * items. Special cases the select-all (SELECT *) and select none
+ * (SELECT COUNT(*)) cases.
+ * <p>
+ * These tests should verify everything about (runtime) projection
+ * parsing; the only bits not tested here is that which is
+ * inherently specific to some use case.
+ */
+@Category(EvfTests.class)
+public class TestProjectionParser extends BaseTest {
+
+  /**
+   * Null map means everything is projected
+   */
+  @Test
+  public void testProjectionAll() {
+    ProjectionParseResult result = ScanProjectionParser.parse(null);
+    assertEquals(0, result.wildcardPosn);
+    TupleMetadata projSet = result.dynamicSchema;
+    assertTrue(SchemaUtils.isProjectAll(projSet));
+    assertFalse(SchemaUtils.isProjectNone(projSet));
+    assertTrue(projSet.isEmpty());
+  }
+
+  /**
+   * SELECT * means everything is projected
+   */
+  @Test
+  public void testWildcard() {
+    ProjectionParseResult result = ScanProjectionParser.parse(
+        RowSetTestUtils.projectAll());
+    assertEquals(0, result.wildcardPosn);
+    TupleMetadata projSet = result.dynamicSchema;
+    assertTrue(SchemaUtils.isProjectAll(projSet));
+    assertFalse(SchemaUtils.isProjectNone(projSet));
+    assertTrue(projSet.isEmpty());
+  }
+
+  /**
+   * Test an empty projection which occurs in a
+   * SELECT COUNT(*) query.
+   * Empty list means nothing is projected.
+   */
+  @Test
+  public void testProjectionNone() {
+    ProjectionParseResult result = ScanProjectionParser.parse(
+        Collections.emptyList());
+    assertEquals(-1, result.wildcardPosn);
+    TupleMetadata projSet = result.dynamicSchema;
+    assertFalse(SchemaUtils.isProjectAll(projSet));
+    assertTrue(SchemaUtils.isProjectNone(projSet));
+    assertTrue(projSet.isEmpty());
+  }
+
+  /**
+   * Simple non-map columns
+   */
+  @Test
+  public void testProjectionSimple() {
+    ProjectionParseResult result = ScanProjectionParser.parse(
+        RowSetTestUtils.projectList("a", "b", "c"));
+    assertEquals(-1, result.wildcardPosn);
+    TupleMetadata projSet = result.dynamicSchema;
+    assertFalse(SchemaUtils.isProjectAll(projSet));
+    assertFalse(SchemaUtils.isProjectNone(projSet));
+    assertNotNull(projSet.metadata("a"));
+    assertNotNull(projSet.metadata("b"));
+    assertNotNull(projSet.metadata("c"));
+    assertNull(projSet.metadata("d"));
+    assertEquals(3, projSet.size());
+
+    ProjectedColumn a = (ProjectedColumn) projSet.metadata(0);
+    assertEquals("a", a.name());
+    assertTrue(a.isSimple());
+    assertFalse(a.isArray());
+    assertFalse(a.isMap());
+    assertNull(a.tupleSchema());
+  }
+
+  /**
+   * The projection set does not enforce uniqueness.
+   */
+  @Test
+  public void testSimpleDups() {
+    ProjectionParseResult result = ScanProjectionParser.parse(
+        RowSetTestUtils.projectList("a", "b", "a"));
+    assertEquals(-1, result.wildcardPosn);
+    TupleMetadata projSet = result.dynamicSchema;
+    assertEquals(2, projSet.size());
+    assertEquals(2, ((ProjectedColumn) projSet.metadata("a")).refCount());
+  }
+
+  /**
+   * Selected map projection, multiple levels, full projection
+   * at leaf level.
+   */
+  @Test
+  public void testProjectionMapSubset() {
+    ProjectionParseResult result = ScanProjectionParser.parse(
+        RowSetTestUtils.projectList("map.a", "map.b", "map.map2.x"));
+    TupleMetadata projSet = result.dynamicSchema;
+
+    // Map itself is projected and has a map qualifier
+    ProjectedColumn map = (ProjectedColumn) projSet.metadata("map");
+    assertNotNull(map);
+
+    // Map: an explicit map-like at top level
+    assertTrue(map.isMap());
+    TupleMetadata mapProj = map.tupleSchema();
+    assertNotNull(mapProj);
+    assertFalse(SchemaUtils.isProjectAll(mapProj));
+    assertFalse(SchemaUtils.isProjectNone(mapProj));
+
+    assertNotNull(mapProj.metadata("a"));
+    assertNotNull(mapProj.metadata("b"));
+    assertNotNull(mapProj.metadata("map2"));
+    assertNull(mapProj.metadata("bogus"));
+
+    // Map b: an implied nested map
+    ProjectedColumn b = (ProjectedColumn) mapProj.metadata("b");
+    assertTrue(b.isSimple());
+
+    // Map2, an nested map, has an explicit projection
+    ProjectedColumn map2 = (ProjectedColumn) mapProj.metadata("map2");
+    TupleMetadata map2Proj = map2.tupleSchema();
+    assertNotNull(map2Proj);
+    assertFalse(SchemaUtils.isProjectAll(map2Proj));
+    assertNotNull(map2Proj.metadata("x"));
+    assertNull(map2Proj.metadata("bogus"));
+  }
+
+  /**
+   * Project both a map member and the entire map.
+   */
+  @Test
+  public void testProjectionMapAndSimple() {
+    ProjectionParseResult result = ScanProjectionParser.parse(
+        RowSetTestUtils.projectList("map.a", "map"));
+    doTestMapAndSimple(result.dynamicSchema);
+  }
+
+  /**
+   * Project both an entire map and a map member.
+   */
+  @Test
+  public void testProjectionSimpleAndMap() {
+    ProjectionParseResult result = ScanProjectionParser.parse(
+        RowSetTestUtils.projectList("map", "map.a"));
+    doTestMapAndSimple(result.dynamicSchema);
+  }
+
+  private void doTestMapAndSimple(TupleMetadata projSet) {
+    ProjectedColumn map = (ProjectedColumn) projSet.metadata("map");
+    assertNotNull(map);
+    assertTrue(map.isMap());
+    TupleMetadata mapProj = map.tupleSchema();
+    assertNotNull(mapProj);
+    assertTrue(SchemaUtils.isProjectAll(mapProj));
+    assertNotNull(mapProj.metadata("a"));
+    assertNull(mapProj.metadata("b"));
+  }
+
+  /**
+   * Project both a map member and the entire map.
+   */
+  @Test
+  public void testProjectionMapAndWildcard() {
+
+    // Built up by hand because "map.*" is not valid Drill
+    // expression syntax.
+    List<SchemaPath> projCols = new ArrayList<>();
+    projCols.add(SchemaPath.getCompoundPath("map", "a"));
+    projCols.add(SchemaPath.getCompoundPath("map", SchemaPath.DYNAMIC_STAR));
+
+    ProjectionParseResult result = ScanProjectionParser.parse(projCols);
+    assertEquals(-1, result.wildcardPosn);
+    TupleMetadata projSet = result.dynamicSchema;
+    doTestMapAndSimple(projSet);
+  }
+
+  /**
+   * Project both an entire map and a map member.
+   */
+  @Test
+  public void testProjectionWildcardAndMap() {
+
+    List<SchemaPath> projCols = new ArrayList<>();
+    projCols.add(SchemaPath.getCompoundPath("map", SchemaPath.DYNAMIC_STAR));
+    projCols.add(SchemaPath.getCompoundPath("map", "a"));
+
+    ProjectionParseResult result = ScanProjectionParser.parse(projCols);
+    assertEquals(-1, result.wildcardPosn);
+    TupleMetadata projSet = result.dynamicSchema;
+    doTestMapAndSimple(projSet);
+  }
+
+  @Test
+  public void testMapDetails() {
+    ProjectionParseResult result = ScanProjectionParser.parse(
+        RowSetTestUtils.projectList("a.b.c", "a.c", "d"));
+    TupleMetadata projSet = result.dynamicSchema;
+    assertEquals(2, projSet.size());
+
+    ProjectedColumn a = (ProjectedColumn) projSet.metadata(0);
+    assertEquals("a", a.name());
+    assertFalse(a.isSimple());
+    assertFalse(a.isArray());
+    assertTrue(a.isMap());
+
+    // a{}
+    TupleMetadata aMembers = a.tupleSchema();
+    assertNotNull(aMembers);
+    assertEquals(2, aMembers.size());
+
+    // a.b
+    ProjectedColumn a_b = (ProjectedColumn) aMembers.metadata(0);
+    assertEquals("b", a_b.name());
+    assertTrue(a_b.isMap());
+
+    // a.b{}
+    TupleMetadata a_bMembers = a_b.tupleSchema();
+    assertNotNull(a_bMembers);
+    assertEquals(1, a_bMembers.size());
+
+    // a.b.c
+    ProjectedColumn a_b_c = (ProjectedColumn) a_bMembers.metadata(0);
+    assertNotNull(a_b_c);
+    assertEquals("c", a_b_c.name());
+    assertTrue(a_b_c.isSimple());
+
+    // a.c
+    ProjectedColumn a_c = (ProjectedColumn) aMembers.metadata(1);
+    assertNotNull(a_c);
+    assertEquals("c", a_c.name());
+    assertTrue(a_c.isSimple());
+
+    // d
+    ProjectedColumn d = (ProjectedColumn) projSet.metadata(1);
+    assertNotNull(d);
+    assertEquals("d", d.name());
+    assertTrue(d.isSimple());
+  }
+
+  /**
+   * Duplicate column names are merged for projection.
+   */
+  @Test
+  public void testMapDups() {
+    ProjectionParseResult result = ScanProjectionParser.parse(
+        RowSetTestUtils.projectList("a.b", "a.c", "a.b"));
+    TupleMetadata projSet = result.dynamicSchema;
+
+    ProjectedColumn a = (ProjectedColumn) projSet.metadata("a");
+    TupleMetadata aMap = a.tupleSchema();
+    assertEquals(2, aMap.size());
+    assertEquals(2, ((ProjectedColumn) aMap.metadata("b")).refCount());
+  }
+
+  @Test
+  public void testArray() {
+    ProjectionParseResult result = ScanProjectionParser.parse(
+        RowSetTestUtils.projectList("a[1]", "a[3]"));
+    TupleMetadata projSet = result.dynamicSchema;
+    assertEquals(1, projSet.size());
+
+    ProjectedColumn a = (ProjectedColumn) projSet.metadata(0);
+    assertEquals("a", a.name());
+    assertTrue(a.isArray());
+    assertEquals(1, a.arrayDims());
+    assertFalse(a.isSimple());
+    assertFalse(a.isMap());
+    assertTrue(a.hasIndexes());
+    boolean indexes[] = a.indexes();
+    assertNotNull(indexes);
+    assertEquals(4, indexes.length);
+    assertFalse(indexes[0]);
+    assertTrue(indexes[1]);
+    assertFalse(indexes[2]);
+    assertTrue(indexes[3]);
+  }
+
+  @Test
+  public void testMultiDimArray() {
+    ProjectionParseResult result = ScanProjectionParser.parse(
+        RowSetTestUtils.projectList("a[0][1][2]", "a[2][3]"));
+    TupleMetadata projSet = result.dynamicSchema;
+    assertEquals(1, projSet.size());
+
+    ProjectedColumn a = (ProjectedColumn) projSet.metadata(0);
+    assertEquals("a", a.name());
+    assertTrue(a.isArray());
+
+    // Dimension count is the maximum seen.
+    assertEquals(3, a.arrayDims());
+    assertFalse(a.isSimple());
+    assertFalse(a.isMap());
+
+    // Indexes only at the first dimension
+    boolean[] indexes = a.indexes();
+    assertNotNull(indexes);
+    assertEquals(3, indexes.length);
+    assertTrue(indexes[0]);
+    assertFalse(indexes[1]);
+    assertTrue(indexes[2]);
+  }
+
+  /**
+   * Duplicate array entries are allowed to handle the
+   * use case of a[1], a[1].z. Each element is reported once;
+   * the project operator will create copies as needed.
+   */
+  @Test
+  public void testArrayDupsIgnored() {
+    ProjectionParseResult result = ScanProjectionParser.parse(
+        RowSetTestUtils.projectList("a[1]", "a[3]", "a[1]", "a[3].z"));
+    TupleMetadata projSet = result.dynamicSchema;
+    assertEquals(1, projSet.size());
+
+    ProjectedColumn a = (ProjectedColumn) projSet.metadata(0);
+    assertEquals("a", a.name());
+    assertTrue(a.isArray());
+    boolean indexes[] = a.indexes();
+    assertNotNull(indexes);
+    assertEquals(4, indexes.length);
+    assertFalse(indexes[0]);
+    assertTrue(indexes[1]);
+    assertFalse(indexes[2]);
+    assertTrue(indexes[3]);
+  }
+
+  @Test
+  public void testArrayAndSimple() {
+    ProjectionParseResult result = ScanProjectionParser.parse(
+        RowSetTestUtils.projectList("a[1]", "a"));
+    doTestArrayAndSimple(result.dynamicSchema);
+  }
+
+  @Test
+  public void testSimpleAndArray() {
+    ProjectionParseResult result = ScanProjectionParser.parse(
+        RowSetTestUtils.projectList("a", "a[1]"));
+    doTestArrayAndSimple(result.dynamicSchema);
+  }
+
+  private void doTestArrayAndSimple(TupleMetadata projSet) {
+    assertEquals(1, projSet.size());
+    ProjectedColumn a = (ProjectedColumn) projSet.metadata(0);
+    assertEquals("a", a.name());
+    assertTrue(a.isArray());
+    assertNull(a.indexes());
+  }
+
+  @Test
+  // Drill syntax does not support map arrays
+  public void testMapArray() {
+    ProjectionParseResult result = ScanProjectionParser.parse(
+        RowSetTestUtils.projectList("a[1].x"));
+    TupleMetadata projSet = result.dynamicSchema;
+    assertEquals(1, projSet.size());
+
+    ProjectedColumn a = (ProjectedColumn) projSet.metadata(0);
+
+    // Column acts like an array
+    assertTrue(a.isArray());
+    assertTrue(a.hasIndexes());
+    assertEquals(1, a.arrayDims());
+
+    // And the column acts like a map
+    assertTrue(a.isMap());
+    TupleMetadata aProj = a.tupleSchema();
+    assertFalse(SchemaUtils.isProjectAll(aProj));
+    assertFalse(SchemaUtils.isProjectNone(aProj));
+    assertNotNull(aProj.metadata("x"));
+    assertNull(aProj.metadata("y"));
+  }
+
+  @Test
+  // Drill syntax does not support map arrays
+  public void testMap2DArray() {
+    ProjectionParseResult result = ScanProjectionParser.parse(
+        RowSetTestUtils.projectList("a[1][2].x"));
+    TupleMetadata projSet = result.dynamicSchema;
+    assertEquals(1, projSet.size());
+
+    ProjectedColumn a = (ProjectedColumn) projSet.metadata(0);
+
+    // Column acts like an array
+    assertTrue(a.isArray());
+    assertTrue(a.hasIndexes());
+
+    // Note that the multiple dimensions are inferred only through
+    // the multiple levels of qualifiers.
+
+    // And the column acts like a map
+    assertTrue(a.isMap());
+    TupleMetadata aProj = a.tupleSchema();
+    assertNotNull(aProj.metadata("x"));
+    assertNull(aProj.metadata("y"));
+  }
+
+  /**
+   * Projection does not enforce semantics; it just report what it
+   * sees. This allows cases such as m.a and m[0], which might mean
+   * that m is a map array, m.a wants an array of a-member values, and m[0]
+   * wants the first map in the array. Not clear Drill actually supports
+   * these cases, however.
+   */
+  @Test
+  public void testArrayAndMap() {
+    ProjectionParseResult result = ScanProjectionParser.parse(
+        RowSetTestUtils.projectList("m.a", "m[0]"));
+    doTestArrayAndMap(result.dynamicSchema);
+  }
+
+  @Test
+  public void testMapAndArray() {
+    ProjectionParseResult result = ScanProjectionParser.parse(
+        RowSetTestUtils.projectList("m[0]", "m.a"));
+    doTestArrayAndMap(result.dynamicSchema);
+  }
+
+  private void doTestArrayAndMap(TupleMetadata projSet) {
+    assertEquals(1, projSet.size());
+    ProjectedColumn m = (ProjectedColumn) projSet.metadata("m");
+    assertTrue(m.isArray());
+    assertEquals(1, m.arrayDims());
+    assertTrue(m.isMap());
+    TupleMetadata mProj = m.tupleSchema();
+    assertNotNull(mProj.metadata("a"));
+    assertNull(mProj.metadata("b"));
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/schema/TestScanSchemaTracker.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/schema/TestScanSchemaTracker.java
new file mode 100644
index 0000000..6ccae71
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/schema/TestScanSchemaTracker.java
@@ -0,0 +1,470 @@
+/*
+ * 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.v3.schema;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.apache.drill.categories.EvfTests;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.EmptyErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.ScanSchemaTracker.ProjectionType;
+import org.apache.drill.exec.physical.resultSet.impl.ProjectionFilter;
+import org.apache.drill.exec.physical.rowSet.RowSetTestUtils;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.test.BaseTest;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test the scan operator schema tracker which computes the final
+ * output schema from a variety of sources.
+ */
+@Category(EvfTests.class)
+public class TestScanSchemaTracker extends BaseTest {
+  private static final CustomErrorContext ERROR_CONTEXT = EmptyErrorContext.INSTANCE;
+
+  /**
+   * Basic test: select a set of columns (a, b, c) when the
+   * data source has an early schema of (a, b).
+   */
+  @Test
+  public void testBasics() {
+
+    // Simulate SELECT a, b, c ...
+    final ScanSchemaConfigBuilder builder = new ScanSchemaConfigBuilder()
+        .projection(RowSetTestUtils.projectList("a", "b", "c"));
+    final ScanSchemaTracker schemaTracker = builder.build();
+
+    assertSame(ProjectionType.SOME, schemaTracker.projectionType());
+    assertFalse(schemaTracker.isResolved());
+    final int initVersion = schemaTracker.schemaVersion();
+    assertTrue(initVersion > 0);
+
+    // Reader input schema is dynamic
+    final TupleMetadata readerInputSchema = schemaTracker.readerInputSchema();
+    final TupleMetadata expected = new SchemaBuilder()
+        .addDynamic("a")
+        .addDynamic("b")
+        .addDynamic("c")
+        .build();
+    assertEquals(expected, readerInputSchema);
+
+    // Pretend the reader discovered two of the columns.
+    final TupleMetadata readerOutputSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .add("b", MinorType.BIGINT)
+        .buildSchema();
+
+    // Projection filter is list-based
+    ProjectionFilter filter = schemaTracker.projectionFilter(ERROR_CONTEXT);
+    assertTrue(filter instanceof DynamicSchemaFilter);
+    assertFalse(filter.isEmpty());
+    assertTrue(filter.projection(readerOutputSchema.metadata("a")).isProjected);
+    assertTrue(filter.projection(readerOutputSchema.metadata("b")).isProjected);
+    assertTrue(filter.isProjected("c"));
+    assertFalse(filter.isProjected("d"));
+    assertFalse(filter.projection(MetadataUtils.newScalar(
+        "d", Types.optional(MinorType.INT))).isProjected);
+
+    schemaTracker.applyReaderSchema(readerOutputSchema, ERROR_CONTEXT);
+    assertFalse(schemaTracker.isResolved());
+    final int reader1Version = schemaTracker.schemaVersion();
+    assertTrue(reader1Version > initVersion);
+
+    // Make up a type for the missing column
+    final TupleMetadata missingCols = schemaTracker.missingColumns(readerOutputSchema);
+    assertEquals(1, missingCols.size());
+    assertEquals(MinorType.LATE, missingCols.metadata("c").type());
+
+    TupleMetadata missingColSchema = new SchemaBuilder()
+        .add("c", MinorType.VARCHAR)
+        .buildSchema();
+    schemaTracker.resolveMissingCols(missingColSchema);
+    assertTrue(schemaTracker.isResolved());
+    final int missing1Version = schemaTracker.schemaVersion();
+    assertTrue(missing1Version > reader1Version);
+
+    // Second reader finds all columns
+    TupleMetadata reader2OutputSchema = new SchemaBuilder()
+        .addAll(readerOutputSchema)
+        .addAll(missingColSchema)
+        .build();
+    schemaTracker.applyReaderSchema(reader2OutputSchema, ERROR_CONTEXT);
+    assertEquals(missing1Version, schemaTracker.schemaVersion());
+
+    // Third reader finds two columns, treats "c" as missing again.
+    schemaTracker.resolveMissingCols(missingColSchema);
+
+    // Final schema sent downstream
+    final TupleMetadata outputSchema = schemaTracker.outputSchema();
+    final TupleMetadata expectedOutput = new SchemaBuilder()
+        .addAll(readerOutputSchema)
+        .addAll(missingColSchema)
+        .buildSchema();
+    assertEquals(expectedOutput, outputSchema);
+  }
+
+  /**
+   * Wildcard projection, schema change allowed.
+   */
+  @Test
+  public void testWildcard() {
+
+    // Simulate SELECT * ...
+    final ScanSchemaConfigBuilder builder = new ScanSchemaConfigBuilder()
+        .projection(RowSetTestUtils.projectAll());
+    final ScanSchemaTracker schemaTracker = builder.build();
+
+    assertSame(ProjectionType.ALL, schemaTracker.projectionType());
+    assertFalse(schemaTracker.isResolved());
+
+    // Reader input schema is dynamic
+    final TupleMetadata reader1InputSchema = schemaTracker.readerInputSchema();
+    assertTrue(reader1InputSchema.isEmpty());
+
+    ProjectionFilter filter1 = schemaTracker.projectionFilter(ERROR_CONTEXT);
+    assertSame(ProjectionFilter.PROJECT_ALL, filter1);
+
+    // Pretend the reader discovers two columns.
+    final TupleMetadata reader1OutputSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .add("b", MinorType.BIGINT)
+        .buildSchema();
+
+    schemaTracker.applyReaderSchema(reader1OutputSchema, ERROR_CONTEXT);
+    assertTrue(schemaTracker.isResolved());
+
+    // Schema sent downstream after first batch
+    final TupleMetadata outputSchema1 = schemaTracker.outputSchema();
+    assertEquals(reader1OutputSchema, outputSchema1);
+
+    // Next reader schema is partially defined
+    final TupleMetadata reader2InputSchema = schemaTracker.readerInputSchema();
+    assertEquals(reader1OutputSchema, reader2InputSchema);
+
+    ProjectionFilter filter2 = schemaTracker.projectionFilter(ERROR_CONTEXT);
+    assertTrue(filter2 instanceof DynamicSchemaFilter);
+    assertTrue(filter2.projection(reader1OutputSchema.metadata("a")).isProjected);
+    assertTrue(filter2.projection(reader1OutputSchema.metadata("b")).isProjected);
+    assertTrue(filter2.isProjected("c"));
+    try {
+      filter2.projection(MetadataUtils.newScalar("a", Types.required(MinorType.VARCHAR)));
+      fail();
+    } catch (UserException e) {
+      // Expected;
+    }
+
+    // The next reader defines another column.
+    // This triggers a schema change in output.
+    final TupleMetadata reader2OutputSchema = new SchemaBuilder()
+        .add("c", MinorType.VARCHAR)
+        .buildSchema();
+    schemaTracker.applyReaderSchema(reader2OutputSchema, ERROR_CONTEXT);
+
+    // Schema sent downstream after second reader
+    final TupleMetadata outputSchema2 = schemaTracker.outputSchema();
+    final TupleMetadata expectedOutput = new SchemaBuilder()
+        .addAll(reader1OutputSchema)
+        .addAll(reader2OutputSchema)
+        .buildSchema();
+    assertEquals(expectedOutput, outputSchema2);
+  }
+
+  /**
+   * The provided schema is a bit like a defined schema, but it is more
+   * of an advisory. The provided schema is completely independent of the
+   * project list: the user is responsible for choosing the project list
+   * wisely.
+   */
+  @Test
+  public void testProvidedSchema() {
+
+    // Simulate SELECT a, b, c ...
+    // With a plan-provided defined schema
+    final ScanSchemaConfigBuilder builder = new ScanSchemaConfigBuilder()
+        .projection(RowSetTestUtils.projectList("a", "b", "c"));
+
+    final TupleMetadata providedSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .add("b", MinorType.BIGINT)
+       .buildSchema();
+    builder.providedSchema(providedSchema);
+
+    final ScanSchemaTracker schemaTracker = builder.build();
+    assertSame(ProjectionType.SOME, schemaTracker.projectionType());
+    assertFalse(schemaTracker.isResolved());
+
+    // Reader input schema is partially defined
+    TupleMetadata readerInputSchema = schemaTracker.readerInputSchema();
+    final TupleMetadata expected = new SchemaBuilder()
+       .addAll(providedSchema)
+       .addDynamic("c")
+       .buildSchema();
+    assertEquals(expected, readerInputSchema);
+
+    // Pretend the reader dutifully provided two of the columns
+    TupleMetadata readerOutputSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .add("c", MinorType.VARCHAR)
+        .buildSchema();
+
+    schemaTracker.applyReaderSchema(readerOutputSchema, ERROR_CONTEXT);
+    assertTrue(schemaTracker.isResolved());
+
+    // Final schema sent downstream
+    final TupleMetadata outputSchema = schemaTracker.outputSchema();
+    final TupleMetadata expectedOutput = new SchemaBuilder()
+        .addAll(providedSchema)
+        .add("c", MinorType.VARCHAR)
+        .buildSchema();
+    assertEquals(expectedOutput, outputSchema);
+  }
+
+  @Test
+  public void testProvidedSchemaWithWildcard() {
+
+    // Simulate SELECT * ...
+    final ScanSchemaConfigBuilder builder = new ScanSchemaConfigBuilder();
+
+    final TupleMetadata providedSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .add("b", MinorType.BIGINT)
+        .buildSchema();
+    builder.providedSchema(providedSchema);
+    final ScanSchemaTracker schemaTracker = builder.build();
+
+    assertSame(ProjectionType.ALL, schemaTracker.projectionType());
+    assertTrue(schemaTracker.isResolved());
+
+    // Reader input schema is dynamic
+    final TupleMetadata reader1InputSchema = schemaTracker.readerInputSchema();
+    assertEquals(providedSchema, reader1InputSchema);
+
+    ProjectionFilter filter1 = schemaTracker.projectionFilter(ERROR_CONTEXT);
+    assertTrue(filter1 instanceof DynamicSchemaFilter);
+
+    // Pretend the reader discovers two columns.
+    final TupleMetadata reader1OutputSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .add("c", MinorType.VARCHAR)
+        .buildSchema();
+    assertTrue(filter1.projection(reader1OutputSchema.metadata("a")).isProjected);
+    assertTrue(filter1.projection(reader1OutputSchema.metadata("c")).isProjected);
+    assertTrue(filter1.isProjected("d"));
+
+    schemaTracker.applyReaderSchema(reader1OutputSchema, ERROR_CONTEXT);
+    assertTrue(schemaTracker.isResolved());
+
+    // Pretend something fills in the the missing columns
+    TupleMetadata missingColSchema = new SchemaBuilder()
+        .add("b", MinorType.BIGINT)
+        .buildSchema();
+    schemaTracker.applyReaderSchema(missingColSchema, ERROR_CONTEXT);
+
+    // Schema sent downstream after first batch
+    final TupleMetadata outputSchema = schemaTracker.outputSchema();
+    final TupleMetadata expected = new SchemaBuilder()
+        .addAll(providedSchema)
+        .add(reader1OutputSchema.metadata("c"))
+        .buildSchema();
+    assertEquals(expected, outputSchema);
+  }
+
+  @Test
+  public void testStrictProvidedSchemaWithWildcard() {
+
+    // Simulate SELECT * ...
+    final ScanSchemaConfigBuilder builder = new ScanSchemaConfigBuilder();
+
+    final TupleMetadata providedSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .add("b", MinorType.BIGINT)
+        .buildSchema();
+    providedSchema.setBooleanProperty(TupleMetadata.IS_STRICT_SCHEMA_PROP, true);
+    builder.providedSchema(providedSchema);
+    final ScanSchemaTracker schemaTracker = builder.build();
+
+    // Even though the project list is a wildcard, the presence
+    // of a strict provided schema makes it fully defined
+    assertSame(ProjectionType.SOME, schemaTracker.projectionType());
+    assertTrue(schemaTracker.isResolved());
+
+    // Reader input schema is fixed
+    final TupleMetadata reader1InputSchema = schemaTracker.readerInputSchema();
+    assertEquals(providedSchema, reader1InputSchema);
+
+    ProjectionFilter filter1 = schemaTracker.projectionFilter(ERROR_CONTEXT);
+    assertTrue(filter1 instanceof DynamicSchemaFilter);
+
+    // Pretend the reader discovers two columns.
+    final TupleMetadata reader1OutputSchema = new SchemaBuilder()
+        .addAll(providedSchema)
+        .buildSchema();
+    assertTrue(filter1.projection(reader1OutputSchema.metadata("a")).isProjected);
+    assertFalse(filter1.isProjected("c"));
+
+    schemaTracker.applyReaderSchema(reader1OutputSchema, ERROR_CONTEXT);
+    assertTrue(schemaTracker.isResolved());
+
+    // Schema sent downstream after first batch
+    final TupleMetadata outputSchema = schemaTracker.outputSchema();
+    assertEquals(providedSchema, outputSchema);
+  }
+
+  /**
+   * Wildcard projection, schema change not allowed; first batch
+   * defines the schema and projection for later readers.
+   */
+  @Test
+  public void testWildcardWithoutSchemaChange() {
+
+    // Simulate SELECT * ...
+    final ScanSchemaConfigBuilder builder = new ScanSchemaConfigBuilder()
+        .projection(RowSetTestUtils.projectAll());
+    builder.allowSchemaChange(false);
+    final ScanSchemaTracker schemaTracker = builder.build();
+
+    assertSame(ProjectionType.ALL, schemaTracker.projectionType());
+    assertFalse(schemaTracker.isResolved());
+
+    // Reader input schema is dynamic
+    ProjectionFilter filter1 = schemaTracker.projectionFilter(ERROR_CONTEXT);
+    assertSame(ProjectionFilter.PROJECT_ALL, filter1);
+    assertTrue(filter1.isProjected("z"));
+
+    TupleMetadata readerInputSchema = schemaTracker.readerInputSchema();
+    assertTrue(readerInputSchema.isEmpty());
+
+    // Pretend the reader discovered two columns.
+    final TupleMetadata reader1OutputSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .add("b", MinorType.BIGINT)
+        .buildSchema();
+
+    schemaTracker.applyReaderSchema(reader1OutputSchema, ERROR_CONTEXT);
+    assertTrue(schemaTracker.isResolved());
+
+    // Next reader schema is completely defined
+    assertSame(ProjectionType.SOME, schemaTracker.projectionType());
+    readerInputSchema = schemaTracker.readerInputSchema();
+    assertEquals(reader1OutputSchema, readerInputSchema);
+
+    // Projection list changes to what we've seen.
+    ProjectionFilter filter2 = schemaTracker.projectionFilter(ERROR_CONTEXT);
+    assertTrue(filter2 instanceof DynamicSchemaFilter);
+    assertTrue(filter2.isProjected("a"));
+    assertTrue(filter2.isProjected("b"));
+    assertFalse(filter2.isProjected("c"));
+
+    schemaTracker.applyReaderSchema(reader1OutputSchema, ERROR_CONTEXT);
+    // Schema sent downstream after first batch
+    TupleMetadata outputSchema = schemaTracker.outputSchema();
+    assertEquals(reader1OutputSchema, outputSchema);
+
+    // The next reader defines another column: "c". The projection filter ignores
+    // the column.
+
+    // The schema tracker ignores this column.
+    final TupleMetadata reader2OutputSchema = new SchemaBuilder()
+        .buildSchema();
+    schemaTracker.applyReaderSchema(reader2OutputSchema, ERROR_CONTEXT);
+
+    // If the reader were to bypass the projection filter and define the
+    // column anyway, the query will fail because the scan framework does
+    // not know what to do with the unwanted (materialized) column.
+    final TupleMetadata reader3OutputSchema = new SchemaBuilder()
+        .add("c", MinorType.VARCHAR)
+        .buildSchema();
+    try {
+      schemaTracker.applyReaderSchema(reader3OutputSchema, ERROR_CONTEXT);
+      fail();
+    } catch (IllegalStateException e) {
+      // Expected;
+    }
+
+    // Schema sent downstream after second reader
+    outputSchema = schemaTracker.outputSchema();
+    assertEquals(reader1OutputSchema, outputSchema);
+  }
+
+  @Test
+  public void testEmptyProject() {
+
+    // Simulate SELECT ...
+    // That is, project nothing, as for COUNT(*)
+    final ScanSchemaConfigBuilder builder = new ScanSchemaConfigBuilder()
+        .projection(RowSetTestUtils.projectNone());
+    doTestEmptyProject(builder);
+  }
+
+  @Test
+  public void testEmptyProjectWithProvidedSchema() {
+
+    // Simulate SELECT ...
+    // That is, project nothing, as for COUNT(*)
+    final ScanSchemaConfigBuilder builder = new ScanSchemaConfigBuilder()
+        .projection(RowSetTestUtils.projectNone());
+    final TupleMetadata providedSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .add("b", MinorType.BIGINT)
+       .buildSchema();
+    builder.providedSchema(providedSchema);
+    doTestEmptyProject(builder);
+  }
+
+  private void doTestEmptyProject(ScanSchemaConfigBuilder builder) {
+    final ScanSchemaTracker schemaTracker = builder.build();
+
+    assertSame(ProjectionType.NONE, schemaTracker.projectionType());
+    assertTrue(schemaTracker.isResolved());
+
+    // Sanity check of projection; detailed testing done elsewhere
+    ProjectionFilter filter1 = schemaTracker.projectionFilter(ERROR_CONTEXT);
+    assertSame(ProjectionFilter.PROJECT_NONE, filter1);
+    assertFalse(filter1.isProjected("a"));
+
+    // Reader input schema is empty
+    TupleMetadata readerInputSchema = schemaTracker.readerInputSchema();
+    assertTrue(readerInputSchema.isEmpty());
+
+    // Projection is empty
+    ProjectionFilter filter = schemaTracker.projectionFilter(ERROR_CONTEXT);
+    assertTrue(filter.isEmpty());
+    assertFalse(filter.isProjected("a"));
+
+    // Reader produces a empty schema
+    final TupleMetadata readerOutputSchema = new SchemaBuilder()
+        .buildSchema();
+
+    schemaTracker.applyReaderSchema(readerOutputSchema, ERROR_CONTEXT);
+    assertTrue(schemaTracker.isResolved());
+
+    // Schema sent downstream after first batch
+    final TupleMetadata outputSchema = schemaTracker.outputSchema();
+    assertTrue(outputSchema.isEmpty());
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/schema/TestScanSchemaTrackerMaps.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/schema/TestScanSchemaTrackerMaps.java
new file mode 100644
index 0000000..c7e4851
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/schema/TestScanSchemaTrackerMaps.java
@@ -0,0 +1,1052 @@
+/*
+ * 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.v3.schema;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.apache.drill.categories.EvfTests;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.EmptyErrorContext;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.resultSet.impl.ProjectionFilter;
+import org.apache.drill.exec.physical.resultSet.impl.ProjectionFilter.ProjResult;
+import org.apache.drill.exec.physical.rowSet.RowSetTestUtils;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.test.BaseTest;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Maps add considerable complexity to the scan schema tracker, turning
+ * a list of columns into a tree. The projection list, provided schema,
+ * defined schema, scan schema, reader schema and missing columns schemas
+ * must all be trees, and must all be kept in sync.
+ */
+@Category(EvfTests.class)
+public class TestScanSchemaTrackerMaps extends BaseTest {
+  private static final CustomErrorContext ERROR_CONTEXT = EmptyErrorContext.INSTANCE;
+
+  private boolean isProjected(ProjectionFilter filter, ColumnMetadata col) {
+    return filter.projection(col).isProjected;
+  }
+
+  private ProjResult mapProjection(ProjectionFilter filter, String mapName) {
+    return filter.projection(MetadataUtils.newMap(mapName));
+  }
+
+  /**
+   * Test a map projected with just the map name: {@code `m`}.
+   */
+  @Test
+  public void testGenericMap() {
+    final ScanSchemaConfigBuilder builder = new ScanSchemaConfigBuilder()
+        .projection(RowSetTestUtils.projectList("a", "m"));
+    doTestGenericMap(builder, true);
+  }
+
+  /**
+   * Test a map projected with the map name and a specific
+   * column: {@code `m`.`x`, `m`}. The map is generic: the entire
+   * map is projected.
+   */
+  @Test
+  public void testGenericAndSpecificMap() {
+    final ScanSchemaConfigBuilder builder = new ScanSchemaConfigBuilder()
+        .projection(RowSetTestUtils.projectList("a", "m.x", "m"));
+    doTestGenericMap(builder, false);
+  }
+
+  private void doTestGenericMap(ScanSchemaConfigBuilder builder, boolean isAll) {
+    final ScanSchemaTracker schemaTracker = builder.build();
+
+    // Pretend the reader discovers that m is a map.
+    TupleMetadata reader1InputSchema = schemaTracker.readerInputSchema();
+    assertEquals(2, reader1InputSchema.size());
+
+    final TupleMetadata readerOutputSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m")
+          .add("x", MinorType.BIGINT)
+          .add("y", MinorType.VARCHAR)
+          .resumeSchema()
+        .buildSchema();
+
+    {
+      ProjectionFilter filter = schemaTracker.projectionFilter(ERROR_CONTEXT);
+      assertTrue(filter instanceof DynamicSchemaFilter);
+      assertTrue(isProjected(filter, readerOutputSchema.metadata("a")));
+      assertTrue(isProjected(filter, readerOutputSchema.metadata("m")));
+      assertFalse(filter.isProjected("b"));
+
+      final ColumnMetadata mapCol = readerOutputSchema.metadata("m");
+      ProjResult mapResult = filter.projection(mapCol);
+      assertTrue(mapResult.isProjected);
+      ProjectionFilter mapFilter = mapResult.mapFilter;
+      if (isAll) {
+        assertSame(ProjectionFilter.PROJECT_ALL, mapFilter);
+      } else {
+        assertTrue(mapFilter instanceof DynamicSchemaFilter);
+      }
+      final TupleMetadata mapSchema = mapCol.tupleSchema();
+      assertTrue(isProjected(mapFilter, mapSchema.metadata("x")));
+      assertTrue(isProjected(mapFilter, mapSchema.metadata("y")));
+      assertTrue(mapFilter.isProjected("z"));
+
+      assertSame(ProjectionFilter.PROJECT_ALL,
+          mapProjection(mapFilter, "w").mapFilter);
+    }
+
+    schemaTracker.applyReaderSchema(readerOutputSchema, ERROR_CONTEXT);
+
+    // Scan output schema is now resolved.
+    assertTrue(schemaTracker.isResolved());
+    final TupleMetadata outputSchema = schemaTracker.outputSchema();
+    assertEquals(readerOutputSchema, outputSchema);
+
+    // A second reader gets a strict filter for the row, but a
+    // project-all filter for the map.
+    TupleMetadata reader2InputSchema = schemaTracker.readerInputSchema();
+    assertEquals(outputSchema, reader2InputSchema);
+
+    {
+      final ProjectionFilter filter = schemaTracker.projectionFilter(ERROR_CONTEXT);
+      assertTrue(filter instanceof DynamicSchemaFilter);
+      assertTrue(isProjected(filter, readerOutputSchema.metadata("a")));
+      assertTrue(isProjected(filter, readerOutputSchema.metadata("m")));
+      assertFalse(filter.isProjected("b"));
+
+      final TupleMetadata mapSchema = readerOutputSchema.metadata("m").tupleSchema();
+      ProjResult mapResult = filter.projection(readerOutputSchema.metadata("m"));
+      assertTrue(mapResult.isProjected);
+      ProjectionFilter mapFilter = mapResult.mapFilter;
+      assertTrue(mapFilter instanceof DynamicSchemaFilter);
+      assertTrue(isProjected(mapFilter, mapSchema.metadata("x")));
+      assertTrue(isProjected(mapFilter, mapSchema.metadata("y")));
+      assertTrue(mapFilter.isProjected("z"));
+
+      assertSame(ProjectionFilter.PROJECT_ALL, mapFilter.projection(
+          MetadataUtils.newMap("w")).mapFilter);
+    }
+  }
+
+  @Test
+  public void testTwoLevelGenericMap() {
+    final ScanSchemaConfigBuilder builder = new ScanSchemaConfigBuilder()
+        .projection(RowSetTestUtils.projectList("a", "m"));
+    final ScanSchemaTracker schemaTracker = builder.build();
+
+    // Pretend the reader discovers that m is a map.
+    final TupleMetadata reader1InputSchema = schemaTracker.readerInputSchema();
+    assertEquals(2, reader1InputSchema.size());
+
+    final TupleMetadata readerOutputSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m")
+          .add("x", MinorType.BIGINT)
+          .add("y", MinorType.VARCHAR)
+          .addMap("m2")
+            .add("p", MinorType.BIGINT)
+            .add("q", MinorType.VARCHAR)
+            .resumeMap()
+          .resumeSchema()
+        .buildSchema();
+
+    {
+      final ProjectionFilter filter = schemaTracker.projectionFilter(ERROR_CONTEXT);
+
+      ProjResult result = filter.projection(MetadataUtils.newMap("m"));
+      assertTrue(result.isProjected);
+      final ProjectionFilter map1Filter = result.mapFilter;
+      assertSame(ProjectionFilter.PROJECT_ALL, map1Filter);
+
+      result = map1Filter.projection(MetadataUtils.newMap("m2"));
+      assertTrue(result.isProjected);
+      assertSame(ProjectionFilter.PROJECT_ALL, result.mapFilter);
+    }
+
+    schemaTracker.applyReaderSchema(readerOutputSchema, ERROR_CONTEXT);
+
+    // Scan output schema is now resolved.
+    assertTrue(schemaTracker.isResolved());
+    assertEquals(readerOutputSchema, schemaTracker.outputSchema());
+
+    // A second reader gets a strict filter for the row, but a
+    // project-all filter for the map.
+    TupleMetadata reader2InputSchema = schemaTracker.readerInputSchema();
+    assertEquals(readerOutputSchema, reader2InputSchema);
+
+    {
+      final ProjectionFilter filter = schemaTracker.projectionFilter(ERROR_CONTEXT);
+      assertTrue(filter instanceof DynamicSchemaFilter);
+
+      final ProjectionFilter map1Filter = mapProjection(filter, "m").mapFilter;
+      assertTrue(map1Filter instanceof DynamicSchemaFilter);
+      assertTrue(map1Filter.isProjected("z"));
+
+      final ProjectionFilter map2Filter = mapProjection(map1Filter, "m2").mapFilter;
+      assertTrue(map2Filter instanceof DynamicSchemaFilter);
+      assertTrue(map2Filter.isProjected("r"));
+    }
+
+    final TupleMetadata reader2OutputSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m")
+          .add("x", MinorType.BIGINT)
+          .add("y", MinorType.VARCHAR)
+          .addMap("m2")
+            .add("p", MinorType.BIGINT)
+            .add("q", MinorType.VARCHAR)
+            .add("r", MinorType.INT)
+            .resumeMap()
+          .add("z", MinorType.VARCHAR)
+          .resumeSchema()
+        .buildSchema();
+    schemaTracker.applyReaderSchema(reader2OutputSchema, ERROR_CONTEXT);
+    assertEquals(reader2OutputSchema, schemaTracker.outputSchema());
+  }
+
+  @Test
+  public void testMapWithWildcard() {
+    final ScanSchemaConfigBuilder builder = new ScanSchemaConfigBuilder();
+    final ScanSchemaTracker schemaTracker = builder.build();
+    assertFalse(schemaTracker.isResolved());
+
+    // Pretend the reader discovers that m is a map.
+    TupleMetadata reader1InputSchema = schemaTracker.readerInputSchema();
+    assertTrue(reader1InputSchema.isEmpty());
+
+    final TupleMetadata readerOutputSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m")
+          .add("x", MinorType.BIGINT)
+          .add("y", MinorType.VARCHAR)
+          .resumeSchema()
+        .buildSchema();
+
+    {
+      final ProjectionFilter filter = schemaTracker.projectionFilter(ERROR_CONTEXT);
+      assertSame(ProjectionFilter.PROJECT_ALL, filter);
+      assertFalse(filter.isEmpty());
+    }
+
+    schemaTracker.applyReaderSchema(readerOutputSchema, ERROR_CONTEXT);
+
+    // Scan output schema is now resolved.
+    assertTrue(schemaTracker.isResolved());
+    final TupleMetadata outputSchema = schemaTracker.outputSchema();
+    assertEquals(readerOutputSchema, outputSchema);
+
+    // A second reader gets a strict filter for the row, but a
+    // project-all filter for the map.
+    TupleMetadata reader2InputSchema = schemaTracker.readerInputSchema();
+    assertEquals(outputSchema, reader2InputSchema);
+
+    {
+      final ProjectionFilter filter = schemaTracker.projectionFilter(ERROR_CONTEXT);
+      assertTrue(filter instanceof DynamicSchemaFilter);
+      assertFalse(filter.isEmpty());
+      assertTrue(isProjected(filter, readerOutputSchema.metadata("a")));
+      assertTrue(isProjected(filter, readerOutputSchema.metadata("m")));
+      assertTrue(filter.isProjected("b"));
+
+      assertSame(ProjectionFilter.PROJECT_ALL, mapProjection(filter, "w").mapFilter);
+
+      final ColumnMetadata mapCol = readerOutputSchema.metadata("m");
+      final ProjectionFilter mapFilter = filter.projection(mapCol).mapFilter;
+      assertTrue(mapFilter instanceof DynamicSchemaFilter);
+      final TupleMetadata mapSchema = mapCol.tupleSchema();
+      assertTrue(isProjected(mapFilter, mapSchema.metadata("x")));
+      assertTrue(isProjected(mapFilter, mapSchema.metadata("y")));
+      assertTrue(mapFilter.isProjected("z"));
+
+      assertSame(ProjectionFilter.PROJECT_ALL,
+          mapProjection(mapFilter, "w").mapFilter);
+    }
+  }
+
+  @Test
+  public void testMapWithWildcardNoSchemaChange() {
+    final ScanSchemaConfigBuilder builder = new ScanSchemaConfigBuilder();
+    builder.allowSchemaChange(false);
+    final ScanSchemaTracker schemaTracker = builder.build();
+    assertFalse(schemaTracker.isResolved());
+
+    // Pretend the reader discovers that m is a map.
+    TupleMetadata reader1InputSchema = schemaTracker.readerInputSchema();
+    assertTrue(reader1InputSchema.isEmpty());
+
+    final TupleMetadata readerOutputSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m")
+          .add("x", MinorType.BIGINT)
+          .add("y", MinorType.VARCHAR)
+          .resumeSchema()
+        .buildSchema();
+
+    {
+      final ProjectionFilter filter = schemaTracker.projectionFilter(ERROR_CONTEXT);
+      assertSame(ProjectionFilter.PROJECT_ALL, filter);
+      assertFalse(filter.isEmpty());
+    }
+
+    schemaTracker.applyReaderSchema(readerOutputSchema, ERROR_CONTEXT);
+
+    // Scan output schema is now resolved.
+    assertTrue(schemaTracker.isResolved());
+    assertEquals(readerOutputSchema, schemaTracker.outputSchema());
+
+    // A second reader gets a strict filter for the row, but a
+    // project-all filter for the map.
+    TupleMetadata reader2InputSchema = schemaTracker.readerInputSchema();
+    assertEquals(readerOutputSchema, reader2InputSchema);
+
+    {
+      final ProjectionFilter filter = schemaTracker.projectionFilter(ERROR_CONTEXT);
+      assertTrue(filter instanceof DynamicSchemaFilter);
+      assertFalse(filter.isEmpty());
+      assertTrue(isProjected(filter, readerOutputSchema.metadata("a")));
+      assertTrue(isProjected(filter, readerOutputSchema.metadata("m")));
+      assertFalse(filter.isProjected("b"));
+
+      assertSame(ProjectionFilter.PROJECT_NONE,
+          mapProjection(filter, "w").mapFilter);
+
+      final ColumnMetadata mapCol = readerOutputSchema.metadata("m");
+      final ProjectionFilter mapFilter = filter.projection(mapCol).mapFilter;
+      assertTrue(mapFilter instanceof DynamicSchemaFilter);
+      final TupleMetadata mapSchema = mapCol.tupleSchema();
+      assertTrue(isProjected(mapFilter, mapSchema.metadata("x")));
+      assertTrue(isProjected(mapFilter, mapSchema.metadata("y")));
+      assertFalse(mapFilter.isProjected("z"));
+
+      assertSame(ProjectionFilter.PROJECT_NONE,
+          mapProjection(mapFilter, "w").mapFilter);
+    }
+  }
+
+  /**
+   * Test a specific map column, {@code `m`.`x`}, with no schema.
+   */
+  @Test
+  public void testSpecificMap() {
+    final ScanSchemaConfigBuilder builder = new ScanSchemaConfigBuilder()
+        .projection(RowSetTestUtils.projectList("a", "m.x"));
+    final ScanSchemaTracker schemaTracker = builder.build();
+
+    // Pretend the reader discovers that m is a map.
+    TupleMetadata reader1InputSchema = schemaTracker.readerInputSchema();
+    assertEquals(2, reader1InputSchema.size());
+
+    final TupleMetadata readerOutputSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m")
+          .add("x", MinorType.BIGINT)
+          .resumeSchema()
+        .buildSchema();
+
+    {
+      ProjectionFilter filter = schemaTracker.projectionFilter(ERROR_CONTEXT);
+      assertTrue(filter instanceof DynamicSchemaFilter);
+      assertTrue(isProjected(filter, readerOutputSchema.metadata("a")));
+      assertTrue(isProjected(filter, readerOutputSchema.metadata("m")));
+      assertFalse(filter.isProjected("b"));
+
+      final ColumnMetadata mapCol = readerOutputSchema.metadata("m");
+      ProjResult mapResult = filter.projection(mapCol);
+      assertTrue(mapResult.isProjected);
+      ProjectionFilter mapFilter = mapResult.mapFilter;
+      assertTrue(mapFilter instanceof DynamicSchemaFilter);
+      final TupleMetadata mapSchema = readerOutputSchema.metadata("m").tupleSchema();
+      assertTrue(isProjected(mapFilter, mapSchema.metadata("x")));
+      assertFalse(mapFilter.isProjected("y"));
+
+      assertSame(ProjectionFilter.PROJECT_NONE,
+          mapProjection(mapFilter, "w").mapFilter);
+    }
+
+    schemaTracker.applyReaderSchema(readerOutputSchema, ERROR_CONTEXT);
+
+    // Scan output schema is now resolved.
+    assertTrue(schemaTracker.isResolved());
+    final TupleMetadata outputSchema = schemaTracker.outputSchema();
+    assertEquals(readerOutputSchema, outputSchema);
+
+    // A second reader gets a strict filter for the row, but a
+    // project-all filter for the map.
+    TupleMetadata reader2InputSchema = schemaTracker.readerInputSchema();
+    assertEquals(outputSchema, reader2InputSchema);
+
+    {
+      ProjectionFilter filter = schemaTracker.projectionFilter(ERROR_CONTEXT);
+      assertTrue(filter instanceof DynamicSchemaFilter);
+      assertTrue(isProjected(filter, readerOutputSchema.metadata("a")));
+      assertTrue(isProjected(filter, readerOutputSchema.metadata("m")));
+      assertFalse(filter.isProjected("b"));
+
+      final ColumnMetadata mapCol = readerOutputSchema.metadata("m");
+      ProjResult mapResult = filter.projection(mapCol);
+      assertTrue(mapResult.isProjected);
+      ProjectionFilter mapFilter = mapResult.mapFilter;
+      assertTrue(mapFilter instanceof DynamicSchemaFilter);
+      final TupleMetadata mapSchema = readerOutputSchema.metadata("m").tupleSchema();
+      assertTrue(isProjected(mapFilter, mapSchema.metadata("x")));
+      assertFalse(mapFilter.isProjected("y"));
+
+      assertSame(ProjectionFilter.PROJECT_NONE,
+          mapProjection(mapFilter, "w").mapFilter);
+    }
+  }
+
+  @Test
+  public void testSpecificMapSubset() {
+    final ScanSchemaConfigBuilder builder = new ScanSchemaConfigBuilder()
+        .projection(RowSetTestUtils.projectList("a", "m.x", "m.y"));
+    final ScanSchemaTracker schemaTracker = builder.build();
+
+    // Pretend the reader discovers that m is a map.
+    TupleMetadata reader1InputSchema = schemaTracker.readerInputSchema();
+    assertEquals(2, reader1InputSchema.size());
+
+    final TupleMetadata readerOutputSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m")
+          .add("x", MinorType.BIGINT)
+          .resumeSchema()
+        .buildSchema();
+
+    {
+      final ProjectionFilter filter = schemaTracker.projectionFilter(ERROR_CONTEXT);
+      assertTrue(filter instanceof DynamicSchemaFilter);
+
+      final ColumnMetadata mapCol = readerOutputSchema.metadata("m");
+      final ProjectionFilter mapFilter = filter.projection(mapCol).mapFilter;
+      assertTrue(mapFilter instanceof DynamicSchemaFilter);
+    }
+
+    schemaTracker.applyReaderSchema(readerOutputSchema, ERROR_CONTEXT);
+
+    // Scan output schema is not yet resolved.
+    assertFalse(schemaTracker.isResolved());
+
+    // Missing columns provided
+    final TupleMetadata missingCols = schemaTracker.missingColumns(readerOutputSchema);
+    final TupleMetadata expectedMissingCols = new SchemaBuilder()
+        .addMap("m")
+          .addDynamic("y")
+          .resumeSchema()
+        .buildSchema();
+    assertEquals(expectedMissingCols, missingCols);
+
+    final TupleMetadata missingColsOutput = new SchemaBuilder()
+        .addMap("m")
+          .addNullable("y", MinorType.VARCHAR)
+          .resumeSchema()
+        .buildSchema();
+    schemaTracker.resolveMissingCols(missingColsOutput);
+
+    // Schema is now resolved
+    assertTrue(schemaTracker.isResolved());
+
+    final TupleMetadata expected = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m")
+          .add("x", MinorType.BIGINT)
+          .addNullable("y", MinorType.VARCHAR)
+          .resumeSchema()
+        .buildSchema();
+    assertEquals(expected, schemaTracker.outputSchema());
+  }
+
+  /**
+   * Test a generic map with a lenient provided schema. The schema
+   * defines those columns which do exist, but allow other map members
+   * to be added.
+   */
+  @Test
+  public void testGenericMapWithLenientProvidedSchema() {
+    final ScanSchemaConfigBuilder builder = new ScanSchemaConfigBuilder()
+        .projection(RowSetTestUtils.projectList("a", "m"));
+    final TupleMetadata providedSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m")
+          .add("x", MinorType.BIGINT)
+          .add("y", MinorType.VARCHAR)
+          .resumeSchema()
+        .buildSchema();
+    builder.providedSchema(providedSchema);
+    final ScanSchemaTracker schemaTracker = builder.build();
+    assertTrue(schemaTracker.isResolved());
+
+    // Pretend the reader reads one of the map columns and discovers a new one.
+    TupleMetadata reader1InputSchema = schemaTracker.readerInputSchema();
+    assertEquals(2, reader1InputSchema.size());
+
+    final TupleMetadata readerOutputSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m")
+          .add("x", MinorType.BIGINT)
+          .add("z", MinorType.VARCHAR)
+          .resumeSchema()
+        .buildSchema();
+
+    {
+      ProjectionFilter filter = schemaTracker.projectionFilter(ERROR_CONTEXT);
+      assertTrue(filter instanceof DynamicSchemaFilter);
+      assertTrue(isProjected(filter, readerOutputSchema.metadata("a")));
+      assertTrue(isProjected(filter, readerOutputSchema.metadata("m")));
+      assertFalse(filter.isProjected("b"));
+
+      final ColumnMetadata mapCol = readerOutputSchema.metadata("m");
+      ProjectionFilter mapFilter = filter.projection(mapCol).mapFilter;
+      assertTrue(mapFilter instanceof DynamicSchemaFilter);
+      final TupleMetadata mapSchema = mapCol.tupleSchema();
+      assertTrue(isProjected(mapFilter, mapSchema.metadata("x")));
+      assertTrue(mapFilter.isProjected("y"));
+      assertTrue(isProjected(mapFilter, mapSchema.metadata("z")));
+
+      assertSame(ProjectionFilter.PROJECT_ALL,
+          mapProjection(mapFilter, "w").mapFilter);
+    }
+
+    schemaTracker.applyReaderSchema(readerOutputSchema, ERROR_CONTEXT);
+
+    // Scan output schema is now resolved.
+    assertTrue(schemaTracker.isResolved());
+    final TupleMetadata expectedOutput = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m")
+          .add("x", MinorType.BIGINT)
+          .add("y", MinorType.VARCHAR)
+          .add("z", MinorType.VARCHAR)
+          .resumeSchema()
+        .buildSchema();
+    assertEquals(expectedOutput, schemaTracker.outputSchema());
+
+    // A second reader gets a strict filter for the row, but a
+    // project-all filter for the map.
+    TupleMetadata reader2InputSchema = schemaTracker.readerInputSchema();
+    assertEquals(expectedOutput, reader2InputSchema);
+
+    {
+      ProjectionFilter filter = schemaTracker.projectionFilter(ERROR_CONTEXT);
+      assertTrue(filter instanceof DynamicSchemaFilter);
+      assertTrue(isProjected(filter, readerOutputSchema.metadata("a")));
+      assertTrue(isProjected(filter, readerOutputSchema.metadata("m")));
+      assertFalse(filter.isProjected("b"));
+
+      final ColumnMetadata mapCol = readerOutputSchema.metadata("m");
+      ProjectionFilter mapFilter = filter.projection(mapCol).mapFilter;
+      assertTrue(mapFilter instanceof DynamicSchemaFilter);
+      final TupleMetadata mapSchema = mapCol.tupleSchema();
+      assertTrue(isProjected(mapFilter, mapSchema.metadata("x")));
+      assertTrue(mapFilter.isProjected("w"));
+
+      assertSame(ProjectionFilter.PROJECT_ALL,
+          mapProjection(mapFilter, "w").mapFilter);
+    }
+  }
+
+  /**
+   * Test a generic map with a strict provided schema. The schema
+   * defines those columns which exist, and forbids adding other map
+   * members.
+   */
+  @Test
+  public void testGenericMapWithStrictProvidedSchema() {
+    final ScanSchemaConfigBuilder builder = new ScanSchemaConfigBuilder()
+        .projection(RowSetTestUtils.projectList("a", "m"));
+    final TupleMetadata providedSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m")
+          .add("x", MinorType.BIGINT)
+          .add("y", MinorType.VARCHAR)
+          .resumeSchema()
+        .buildSchema();
+    providedSchema.setBooleanProperty(TupleMetadata.IS_STRICT_SCHEMA_PROP, true);
+    builder.providedSchema(providedSchema);
+    doTestMapDefinedSchema(builder, providedSchema);
+  }
+
+  /**
+   * Test a generic map with a defined schema. The schema
+   * exactly defines the map. Since the map projection is generic,
+   * it matches whatever columns the defined schema defines.
+   */
+  @Test
+  public void testGenericMapWithDefinedSchema() {
+    final ScanSchemaConfigBuilder builder = new ScanSchemaConfigBuilder()
+        .projection(RowSetTestUtils.projectList("a", "m"));
+    final TupleMetadata definedSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m")
+          .add("x", MinorType.BIGINT)
+          .add("y", MinorType.VARCHAR)
+          .resumeSchema()
+        .buildSchema();
+    builder.definedSchema(definedSchema);
+    doTestMapDefinedSchema(builder, definedSchema);
+  }
+
+  private void doTestMapDefinedSchema(ScanSchemaConfigBuilder builder, TupleMetadata targetSchema) {
+    final ScanSchemaTracker schemaTracker = builder.build();
+    assertTrue(schemaTracker.isResolved());
+
+    // Pretend the reader reads one of the map columns and discovers a new one.
+    TupleMetadata reader1InputSchema = schemaTracker.readerInputSchema();
+    assertEquals(targetSchema, reader1InputSchema);
+
+    final TupleMetadata readerOutputSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m")
+          .add("x", MinorType.BIGINT)
+          .resumeSchema()
+        .buildSchema();
+
+    {
+      ProjectionFilter filter = schemaTracker.projectionFilter(ERROR_CONTEXT);
+      assertTrue(filter instanceof DynamicSchemaFilter);
+      assertFalse(filter.isEmpty());
+      assertTrue(isProjected(filter, readerOutputSchema.metadata("a")));
+      assertTrue(isProjected(filter, readerOutputSchema.metadata("m")));
+      assertFalse(filter.isProjected("b"));
+
+      assertSame(ProjectionFilter.PROJECT_NONE,
+          mapProjection(filter, "w").mapFilter);
+
+      final ColumnMetadata mapCol = readerOutputSchema.metadata("m");
+      final ProjectionFilter mapFilter = filter.projection(mapCol).mapFilter;
+      assertTrue(mapFilter instanceof DynamicSchemaFilter);
+      final TupleMetadata mapSchema = mapCol.tupleSchema();
+      assertTrue(isProjected(mapFilter, mapSchema.metadata("x")));
+      assertTrue(mapFilter.isProjected("y"));
+      assertFalse(mapFilter.isProjected("z"));
+
+      assertSame(ProjectionFilter.PROJECT_NONE,
+          mapProjection(mapFilter, "w").mapFilter);
+    }
+
+    schemaTracker.applyReaderSchema(readerOutputSchema, ERROR_CONTEXT);
+
+    // Scan output schema is now resolved.
+    assertTrue(schemaTracker.isResolved());
+    assertEquals(targetSchema, schemaTracker.outputSchema());
+
+    // A second reader gets a strict filter for the row and map.
+    TupleMetadata reader2InputSchema = schemaTracker.readerInputSchema();
+    assertEquals(targetSchema, reader2InputSchema);
+
+    {
+      ProjectionFilter filter = schemaTracker.projectionFilter(ERROR_CONTEXT);
+      assertTrue(filter instanceof DynamicSchemaFilter);
+      assertTrue(isProjected(filter, readerOutputSchema.metadata("a")));
+      assertTrue(isProjected(filter, readerOutputSchema.metadata("m")));
+      assertFalse(filter.isProjected("b"));
+
+      final ColumnMetadata mapCol = readerOutputSchema.metadata("m");
+      final ProjectionFilter mapFilter = filter.projection(mapCol).mapFilter;
+      assertTrue(mapFilter instanceof DynamicSchemaFilter);
+      final TupleMetadata mapSchema = mapCol.tupleSchema();
+      assertTrue(isProjected(mapFilter, mapSchema.metadata("x")));
+      assertFalse(mapFilter.isProjected("w"));
+
+      assertSame(ProjectionFilter.PROJECT_NONE,
+          mapProjection(mapFilter, "w").mapFilter);
+    }
+  }
+
+  /**
+   * Test a specific map column ({@code m.x}) with a lenient
+   * provided schema. The schema can include columns other than
+   * the specific one, but only the specific one will be projected.
+   * Note that the projection list, not the provided schema, constrains
+   * projection in this case.
+   */
+  @Test
+  public void testSpecificMapWithLenientProvidedSchema() {
+    final ScanSchemaConfigBuilder builder = new ScanSchemaConfigBuilder()
+        .projection(RowSetTestUtils.projectList("a", "m.x"));
+    final TupleMetadata providedSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m")
+          .add("x", MinorType.BIGINT)
+          .add("y", MinorType.VARCHAR)
+          .resumeSchema()
+        .buildSchema();
+    builder.providedSchema(providedSchema);
+    final ScanSchemaTracker schemaTracker = builder.build();
+    assertTrue(schemaTracker.isResolved());
+    final int initVersion = schemaTracker.schemaVersion();
+
+    // Pretend the reader reads one of the map columns and discovers a new one.
+    TupleMetadata reader1InputSchema = schemaTracker.readerInputSchema();
+    assertEquals(2, reader1InputSchema.size());
+
+    final TupleMetadata readerOutputSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m")
+          .add("x", MinorType.BIGINT)
+          .resumeSchema()
+        .buildSchema();
+
+    {
+      ProjectionFilter filter = schemaTracker.projectionFilter(ERROR_CONTEXT);
+      assertTrue(filter instanceof DynamicSchemaFilter);
+      assertTrue(isProjected(filter, readerOutputSchema.metadata("a")));
+      assertTrue(isProjected(filter, readerOutputSchema.metadata("m")));
+      assertFalse(filter.isProjected("b"));
+
+      assertSame(ProjectionFilter.PROJECT_NONE,
+          mapProjection(filter, "w").mapFilter);
+
+      final ColumnMetadata mapCol = readerOutputSchema.metadata("m");
+      final ProjectionFilter mapFilter = filter.projection(mapCol).mapFilter;
+      assertTrue(mapFilter instanceof DynamicSchemaFilter);
+      final TupleMetadata mapSchema = mapCol.tupleSchema();
+      assertTrue(isProjected(mapFilter, mapSchema.metadata("x")));
+      assertFalse(mapFilter.isProjected("y"));
+
+      assertSame(ProjectionFilter.PROJECT_NONE,
+          mapProjection(mapFilter, "w").mapFilter);
+    }
+
+    schemaTracker.applyReaderSchema(readerOutputSchema, ERROR_CONTEXT);
+    assertEquals(initVersion, schemaTracker.schemaVersion());
+
+    // Scan output sent downstream
+    assertTrue(schemaTracker.isResolved());
+    assertEquals(readerOutputSchema, schemaTracker.outputSchema());
+  }
+
+  /**
+   * Test a specific map column ({@code m.x}) with a strict
+   * provided schema. The schema can include columns other than
+   * the specific one, but only the specific one will be projected.
+   */
+  @Test
+  public void testSpecificMapWithStrictProvidedSchema() {
+    final ScanSchemaConfigBuilder builder = new ScanSchemaConfigBuilder()
+        .projection(RowSetTestUtils.projectList("a", "m.x"));
+    final TupleMetadata providedSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m")
+          .add("x", MinorType.BIGINT)
+          .add("y", MinorType.VARCHAR)
+          .resumeSchema()
+        .buildSchema();
+    providedSchema.setBooleanProperty(TupleMetadata.IS_STRICT_SCHEMA_PROP, true);
+    builder.providedSchema(providedSchema);
+    final TupleMetadata expected = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m")
+          .add("x", MinorType.BIGINT)
+          .resumeSchema()
+        .buildSchema();
+    doTestSpecificMapWithSchema(builder, expected);
+  }
+
+  /**
+   * Test a specific map column ({@code m.x}) with a defined schema.
+   * The defined schema must exactly match the projection list (because
+   * it should have been computed from that list.)
+   */
+  @Test
+  public void testSpecificMapWithDefinedSchema() {
+    final ScanSchemaConfigBuilder builder = new ScanSchemaConfigBuilder()
+        .projection(RowSetTestUtils.projectList("a", "m.x"));
+    final TupleMetadata definedSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m")
+          .add("x", MinorType.BIGINT)
+          .resumeSchema()
+        .buildSchema();
+    definedSchema.setBooleanProperty(TupleMetadata.IS_STRICT_SCHEMA_PROP, true);
+    builder.definedSchema(definedSchema);
+    doTestSpecificMapWithSchema(builder, definedSchema);
+  }
+
+  private void doTestSpecificMapWithSchema(ScanSchemaConfigBuilder builder, TupleMetadata targetSchema) {
+    final ScanSchemaTracker schemaTracker = builder.build();
+    assertTrue(schemaTracker.isResolved());
+    final int initVersion = schemaTracker.schemaVersion();
+
+    // Pretend the reader reads one of the map columns and discovers a new one.
+    TupleMetadata reader1InputSchema = schemaTracker.readerInputSchema();
+    assertEquals(2, reader1InputSchema.size());
+
+    final TupleMetadata readerOutputSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m")
+          .add("x", MinorType.BIGINT)
+          .resumeSchema()
+        .buildSchema();
+
+    {
+      ProjectionFilter filter = schemaTracker.projectionFilter(ERROR_CONTEXT);
+      assertTrue(filter instanceof DynamicSchemaFilter);
+      assertTrue(isProjected(filter, readerOutputSchema.metadata("a")));
+      assertTrue(isProjected(filter, readerOutputSchema.metadata("m")));
+      assertFalse(filter.isProjected("b"));
+
+      final ColumnMetadata mapCol = readerOutputSchema.metadata("m");
+      final ProjectionFilter mapFilter = filter.projection(mapCol).mapFilter;
+      assertTrue(mapFilter instanceof DynamicSchemaFilter);
+      final TupleMetadata mapSchema = mapCol.tupleSchema();
+      assertTrue(isProjected(mapFilter, mapSchema.metadata("x")));
+      assertFalse(mapFilter.isProjected("y"));
+
+      assertSame(ProjectionFilter.PROJECT_NONE,
+          mapProjection(mapFilter, "w").mapFilter);
+    }
+
+    schemaTracker.applyReaderSchema(readerOutputSchema, ERROR_CONTEXT);
+    assertEquals(initVersion, schemaTracker.schemaVersion());
+
+    // Scan output sent downstream
+    assertTrue(schemaTracker.isResolved());
+    assertEquals(targetSchema, schemaTracker.outputSchema());
+  }
+
+  @Test
+  public void testDynamicMapWithDefinedSchema() {
+    final ScanSchemaConfigBuilder builder = new ScanSchemaConfigBuilder()
+        .projection(RowSetTestUtils.projectList("a", "m"));
+    final TupleMetadata definedSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addDynamic("m")
+        .buildSchema();
+    definedSchema.setBooleanProperty(TupleMetadata.IS_STRICT_SCHEMA_PROP, true);
+    builder.definedSchema(definedSchema);
+    final ScanSchemaTracker schemaTracker = builder.build();
+    assertFalse(schemaTracker.isResolved());
+
+    // Pretend the reader reads one of the map columns and discovers a new one.
+    TupleMetadata reader1InputSchema = schemaTracker.readerInputSchema();
+    assertEquals(2, reader1InputSchema.size());
+
+    final TupleMetadata readerOutputSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m")
+          .add("x", MinorType.BIGINT)
+          .resumeSchema()
+        .buildSchema();
+
+    {
+      ProjectionFilter filter = schemaTracker.projectionFilter(ERROR_CONTEXT);
+      assertTrue(filter instanceof DynamicSchemaFilter);
+      assertTrue(isProjected(filter, readerOutputSchema.metadata("a")));
+      assertTrue(isProjected(filter, readerOutputSchema.metadata("m")));
+      assertFalse(filter.isProjected("b"));
+
+      final ColumnMetadata mapCol = readerOutputSchema.metadata("m");
+      final ProjectionFilter mapFilter = filter.projection(mapCol).mapFilter;
+      assertSame(ProjectionFilter.PROJECT_ALL, mapFilter);
+      final TupleMetadata mapSchema = mapCol.tupleSchema();
+      assertTrue(isProjected(mapFilter, mapSchema.metadata("x")));
+      assertTrue(mapFilter.isProjected("y"));
+
+      assertSame(ProjectionFilter.PROJECT_ALL,
+          mapProjection(mapFilter, "w").mapFilter);
+    }
+
+    schemaTracker.applyReaderSchema(readerOutputSchema, ERROR_CONTEXT);
+
+    // Scan output sent downstream
+    assertTrue(schemaTracker.isResolved());
+    assertEquals(readerOutputSchema, schemaTracker.outputSchema());
+  }
+
+  /**
+   * Test a generic map column ({@code m.x}) with a defined schema.
+   * The defined schema must exactly match the projection list (because
+   * it should have been computed from that list.)
+   */
+  @Test
+  public void testGenericMapWithStrictSchema() {
+    final ScanSchemaConfigBuilder builder = new ScanSchemaConfigBuilder()
+        .projection(RowSetTestUtils.projectList("a", "m"));
+    final TupleMetadata definedSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m")
+          .add("x", MinorType.BIGINT)
+          .resumeSchema()
+        .buildSchema();
+    definedSchema.setBooleanProperty(TupleMetadata.IS_STRICT_SCHEMA_PROP, true);
+    builder.providedSchema(definedSchema);
+    doTestGenericMapWithSchema(builder, definedSchema);
+  }
+
+  @Test
+  public void testDynamicMapWithStrictSchema() {
+    final ScanSchemaConfigBuilder builder = new ScanSchemaConfigBuilder()
+        .projection(RowSetTestUtils.projectList("a", "m"));
+    final TupleMetadata definedSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m")
+          .add("x", MinorType.BIGINT)
+          .resumeSchema()
+        .buildSchema();
+    definedSchema.setBooleanProperty(TupleMetadata.IS_STRICT_SCHEMA_PROP, true);
+    builder.providedSchema(definedSchema);
+    doTestGenericMapWithSchema(builder, definedSchema);
+  }
+
+  private void doTestGenericMapWithSchema(ScanSchemaConfigBuilder builder, TupleMetadata targetSchema) {
+    final ScanSchemaTracker schemaTracker = builder.build();
+    assertTrue(schemaTracker.isResolved());
+    final int initVersion = schemaTracker.schemaVersion();
+
+    // Pretend the reader reads one of the map columns and discovers a new one.
+    TupleMetadata reader1InputSchema = schemaTracker.readerInputSchema();
+    assertEquals(2, reader1InputSchema.size());
+
+    final TupleMetadata readerOutputSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m")
+          .add("x", MinorType.BIGINT)
+          .resumeSchema()
+        .buildSchema();
+
+    {
+      ProjectionFilter filter = schemaTracker.projectionFilter(ERROR_CONTEXT);
+      assertTrue(filter instanceof DynamicSchemaFilter);
+      assertTrue(isProjected(filter, readerOutputSchema.metadata("a")));
+      assertTrue(isProjected(filter, readerOutputSchema.metadata("m")));
+      assertFalse(filter.isProjected("b"));
+
+      final ColumnMetadata mapCol = readerOutputSchema.metadata("m");
+      final ProjectionFilter mapFilter = filter.projection(mapCol).mapFilter;
+      assertTrue(mapFilter instanceof DynamicSchemaFilter);
+      final TupleMetadata mapSchema = mapCol.tupleSchema();
+      assertTrue(isProjected(mapFilter, mapSchema.metadata("x")));
+      assertFalse(mapFilter.isProjected("y"));
+
+      assertSame(ProjectionFilter.PROJECT_NONE,
+          mapProjection(mapFilter, "w").mapFilter);
+    }
+
+    schemaTracker.applyReaderSchema(readerOutputSchema, ERROR_CONTEXT);
+    assertEquals(initVersion, schemaTracker.schemaVersion());
+
+    // Scan output sent downstream
+    assertTrue(schemaTracker.isResolved());
+    assertEquals(targetSchema, schemaTracker.outputSchema());
+  }
+
+  @Test
+  public void testGenericMapWithDynamicSchema() {
+    final ScanSchemaConfigBuilder builder = new ScanSchemaConfigBuilder()
+        .projection(RowSetTestUtils.projectList("a", "m"));
+    final TupleMetadata definedSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addDynamic("m")
+        .buildSchema();
+    definedSchema.setBooleanProperty(TupleMetadata.IS_STRICT_SCHEMA_PROP, true);
+    builder.definedSchema(definedSchema);
+    final ScanSchemaTracker schemaTracker = builder.build();
+    assertFalse(schemaTracker.isResolved());
+    final int initVersion = schemaTracker.schemaVersion();
+
+    // Pretend the reader reads one of the map columns and discovers a new one.
+    TupleMetadata reader1InputSchema = schemaTracker.readerInputSchema();
+    assertEquals(2, reader1InputSchema.size());
+
+    final TupleMetadata readerOutputSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m")
+          .add("x", MinorType.BIGINT)
+          .resumeSchema()
+        .buildSchema();
+
+    {
+      ProjectionFilter filter = schemaTracker.projectionFilter(ERROR_CONTEXT);
+      assertTrue(filter instanceof DynamicSchemaFilter);
+      assertTrue(isProjected(filter, readerOutputSchema.metadata("a")));
+      assertTrue(isProjected(filter, readerOutputSchema.metadata("m")));
+      assertFalse(filter.isProjected("b"));
+
+      final ColumnMetadata mapCol = readerOutputSchema.metadata("m");
+      final ProjectionFilter mapFilter = filter.projection(mapCol).mapFilter;
+      assertSame(ProjectionFilter.PROJECT_ALL, mapFilter);
+    }
+
+    schemaTracker.applyReaderSchema(readerOutputSchema, ERROR_CONTEXT);
+    assertTrue(initVersion < schemaTracker.schemaVersion());
+
+    // Scan output sent downstream
+    assertTrue(schemaTracker.isResolved());
+    assertEquals(readerOutputSchema, schemaTracker.outputSchema());
+  }
+
+  /**
+   * When a defined schema is given, the map projection list
+   * must match the defined schema. Unlike a provided schema,
+   * extra columns are not allowed.
+   */
+  @Test
+  public void testMapProjectionMismatchLength() {
+    final ScanSchemaConfigBuilder builder = new ScanSchemaConfigBuilder()
+        .projection(RowSetTestUtils.projectList("a", "m.x"));
+    final TupleMetadata definedSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m")
+          .add("x", MinorType.BIGINT)
+          .add("y", MinorType.VARCHAR)
+          .resumeSchema()
+        .buildSchema();
+    definedSchema.setBooleanProperty(TupleMetadata.IS_STRICT_SCHEMA_PROP, true);
+    builder.definedSchema(definedSchema);
+    try {
+      builder.build();
+      fail();
+    } catch (IllegalArgumentException e) {
+      // Expected
+    }
+  }
+
+  /**
+   * When a defined schema is given, the map projection list
+   * must match the defined schema. Unlike a provided schema,
+   * disjoint columns are not allowed.
+   */
+  @Test
+  public void testMapProjectionMismatchMembers() {
+    final ScanSchemaConfigBuilder builder = new ScanSchemaConfigBuilder()
+        .projection(RowSetTestUtils.projectList("a", "m.x"));
+    final TupleMetadata definedSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m")
+          .add("y", MinorType.VARCHAR)
+          .resumeSchema()
+        .buildSchema();
+    definedSchema.setBooleanProperty(TupleMetadata.IS_STRICT_SCHEMA_PROP, true);
+    builder.definedSchema(definedSchema);
+    try {
+      builder.build();
+      fail();
+    } catch (IllegalArgumentException e) {
+      // Expected
+    }
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/schema/TestSchemaTrackerDefined.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/schema/TestSchemaTrackerDefined.java
new file mode 100644
index 0000000..76aba7f
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/schema/TestSchemaTrackerDefined.java
@@ -0,0 +1,276 @@
+/*
+ * 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.v3.schema;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.apache.drill.categories.EvfTests;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.EmptyErrorContext;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.ScanSchemaTracker.ProjectionType;
+import org.apache.drill.exec.physical.resultSet.impl.ProjectionFilter;
+import org.apache.drill.exec.physical.rowSet.RowSetTestUtils;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.record.metadata.TupleSchema;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(EvfTests.class)
+public class TestSchemaTrackerDefined {
+  private static final CustomErrorContext ERROR_CONTEXT = EmptyErrorContext.INSTANCE;
+
+  private boolean isProjected(ProjectionFilter filter, ColumnMetadata col) {
+    return filter.projection(col).isProjected;
+  }
+
+  /**
+   * If a schema is defined, then the planner has combined the projection
+   * list with schema information to produce the final output schema
+   * at plan time.
+   */
+  @Test
+  public void testDefinedSchema() {
+
+    // Simulate SELECT a, b, c ...
+    // With a plan-provided defined schema
+    final ScanSchemaConfigBuilder builder = new ScanSchemaConfigBuilder()
+        .projection(RowSetTestUtils.projectList("a", "b", "c"));
+
+    final TupleMetadata definedSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .add("b", MinorType.BIGINT)
+        .add("c", MinorType.VARCHAR)
+       .buildSchema();
+    builder.definedSchema(definedSchema);
+
+    final ScanSchemaTracker schemaTracker = builder.build();
+    assertTrue(schemaTracker instanceof SchemaBasedTracker);
+    assertTrue(schemaTracker.isResolved());
+    assertSame(ProjectionType.SOME, schemaTracker.projectionType());
+
+    // Reader input schema is fully defined
+    TupleMetadata readerInputSchema = schemaTracker.readerInputSchema();
+    assertEquals(definedSchema, readerInputSchema);
+
+    // Pretend the reader dutifully provided two of the columns
+    TupleMetadata readerOutputSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .add("b", MinorType.BIGINT)
+        .buildSchema();
+
+    // Projection filter is schema-based
+    ProjectionFilter filter = schemaTracker.projectionFilter(ERROR_CONTEXT);
+    assertTrue(filter instanceof DynamicSchemaFilter);
+    assertTrue(isProjected(filter, readerOutputSchema.metadata("a")));
+    assertTrue(isProjected(filter, readerOutputSchema.metadata("b")));
+    assertTrue(filter.isProjected("c"));
+    assertFalse(filter.isProjected("d"));
+
+    schemaTracker.applyReaderSchema(readerOutputSchema, ERROR_CONTEXT);
+
+    // Reader schema is still fully resolved
+    assertTrue(schemaTracker.isResolved());
+    assertEquals(definedSchema, readerInputSchema);
+
+    // Pretend a class fills in the the missing columns
+    TupleMetadata missingColSchema = new SchemaBuilder()
+        .add("c", MinorType.VARCHAR)
+        .buildSchema();
+    schemaTracker.applyReaderSchema(missingColSchema, ERROR_CONTEXT);
+    assertEquals(definedSchema, readerInputSchema);
+
+    // Final schema sent downstream
+    final TupleMetadata outputSchema = schemaTracker.outputSchema();
+    assertEquals(definedSchema, outputSchema);
+  }
+
+  /**
+   * If a schema is defined, then the planner has combined the projection
+   * list with schema information to produce the final output schema
+   * at plan time. The planner might leave some columns as dynamic type:
+   * the schema (like projection) says which columns are wanted. But, like
+   * a full dynamic schema, the types may not be known.
+   */
+  @Test
+  public void testDynamicDefinedSchema() {
+
+    // Simulate SELECT a, b, c ...
+    // With a plan-provided defined schema
+    final ScanSchemaConfigBuilder builder = new ScanSchemaConfigBuilder()
+        .projection(RowSetTestUtils.projectList("a", "b", "c"));
+
+    final TupleMetadata definedSchema = new SchemaBuilder()
+        .addDynamic("a")
+        .add("b", MinorType.BIGINT)
+        .add("c", MinorType.VARCHAR)
+       .buildSchema();
+    builder.definedSchema(definedSchema);
+
+    final ScanSchemaTracker schemaTracker = builder.build();
+    assertTrue(schemaTracker instanceof ProjectionSchemaTracker);
+    assertSame(ProjectionType.SOME, schemaTracker.projectionType());
+    assertFalse(schemaTracker.isResolved());
+
+    // Reader input schema is partially defined
+    final TupleMetadata reader1InputSchema = schemaTracker.readerInputSchema();
+    assertEquals(definedSchema, reader1InputSchema);
+
+    ProjectionFilter filter = schemaTracker.projectionFilter(ERROR_CONTEXT);
+    assertTrue(filter instanceof DynamicSchemaFilter);
+
+    // Pretend the reader dutifully provided two of the columns
+    TupleMetadata readerOutputSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .add("b", MinorType.BIGINT)
+        .buildSchema();
+    schemaTracker.applyReaderSchema(readerOutputSchema, ERROR_CONTEXT);
+
+    // Reader schema is now fully resolved
+    assertTrue(schemaTracker.isResolved());
+    final TupleMetadata reader2InputSchema = schemaTracker.readerInputSchema();
+    TupleMetadata expected = new SchemaBuilder()
+        .addAll(readerOutputSchema)
+        .add("c", MinorType.VARCHAR)
+        .build();
+    assertEquals(expected, reader2InputSchema);
+
+    // Pretend an class fills in the the missing columns
+    TupleMetadata missingColSchema = new SchemaBuilder()
+        .add("c", MinorType.VARCHAR)
+        .buildSchema();
+    schemaTracker.applyReaderSchema(missingColSchema, ERROR_CONTEXT);
+
+    // Final schema sent downstream
+    final TupleMetadata outputSchema = schemaTracker.outputSchema();
+    assertEquals(expected, outputSchema);
+  }
+
+  /**
+   * It is an error if the provided schema does not match the projection
+   * list. (Actually, the projection list is redundant in this case. But,
+   * since Drill does not actually support a defined schema yet, we have
+   * to be a bit vague.
+   */
+  @Test
+  public void testTooShortProjection() {
+
+    // Simulate SELECT a, b ...
+    // With a plan-provided defined schema
+    final ScanSchemaConfigBuilder builder = new ScanSchemaConfigBuilder()
+        .projection(RowSetTestUtils.projectList("a", "b"));
+
+    final TupleMetadata definedSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .add("b", MinorType.BIGINT)
+        .add("c", MinorType.VARCHAR)
+       .buildSchema();
+    builder.definedSchema(definedSchema);
+
+    try {
+      builder.build();
+      fail();
+    } catch (Exception e) {
+      // Expected
+    }
+  }
+
+  @Test
+  public void testTooLongProjection() {
+
+    // Simulate SELECT a, b, c, d ...
+    // With a plan-provided defined schema
+    final ScanSchemaConfigBuilder builder = new ScanSchemaConfigBuilder()
+        .projection(RowSetTestUtils.projectList("a", "b", "c", "d"));
+
+    final TupleMetadata definedSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .add("b", MinorType.BIGINT)
+        .add("c", MinorType.VARCHAR)
+       .buildSchema();
+    builder.definedSchema(definedSchema);
+
+    try {
+      builder.build();
+      fail();
+    } catch (Exception e) {
+      // Expected
+    }
+  }
+
+  @Test
+  public void testDisjointProjection() {
+
+    // Simulate SELECT a, c, d ...
+    // With a plan-provided defined schema
+    final ScanSchemaConfigBuilder builder = new ScanSchemaConfigBuilder()
+        .projection(RowSetTestUtils.projectList("a", "c", "d"));
+
+    final TupleMetadata definedSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .add("b", MinorType.BIGINT)
+        .add("c", MinorType.VARCHAR)
+       .buildSchema();
+    builder.definedSchema(definedSchema);
+
+    try {
+      builder.build();
+      fail();
+    } catch (Exception e) {
+      // Expected
+    }
+  }
+
+  @Test
+  public void testDefinedSchemaWildcard() {
+
+    // Simulate SELECT * ...
+    final ScanSchemaConfigBuilder builder = new ScanSchemaConfigBuilder();
+
+    final TupleMetadata definedSchema = new SchemaBuilder()
+        .addDynamic(SchemaPath.DYNAMIC_STAR)
+       .buildSchema();
+    builder.definedSchema(definedSchema);
+    final ScanSchemaTracker schemaTracker = builder.build();
+    assertTrue(schemaTracker instanceof ProjectionSchemaTracker);
+
+    // At this point, the tracker acts just like a non-defined
+    // schema.
+  }
+
+  @Test
+  public void testEmptyProjectWithDefinedSchema() {
+
+    // Simulate SELECT ...
+    // That is, project nothing, as for COUNT(*)
+    final ScanSchemaConfigBuilder builder = new ScanSchemaConfigBuilder()
+        .projection(RowSetTestUtils.projectNone());
+    builder.definedSchema(new TupleSchema());
+    final ScanSchemaTracker schemaTracker = builder.build();
+    assertTrue(schemaTracker instanceof SchemaBasedTracker);
+    assertTrue(schemaTracker.isResolved());
+    assertSame(ProjectionType.NONE, schemaTracker.projectionType());
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/schema/TestSchemaTrackerEarlyReaderSchema.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/schema/TestSchemaTrackerEarlyReaderSchema.java
new file mode 100644
index 0000000..af74205
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/schema/TestSchemaTrackerEarlyReaderSchema.java
@@ -0,0 +1,172 @@
+/*
+ * 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.v3.schema;
+
+import static org.apache.drill.exec.physical.impl.scan.v3.schema.BaseTestSchemaTracker.trackerFor;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.apache.drill.categories.EvfTests;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.EmptyErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.ScanTestUtils;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.ImplicitColumnResolver.ImplicitColumnOptions;
+import org.apache.drill.exec.physical.rowSet.RowSetTestUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.test.SubOperatorTest;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Tests early reader schema against a provided schema.
+ * Basic tests of early reader against a project list are in
+ * {@link TestSchemaTrackerInputSchema}.
+ */
+@Category(EvfTests.class)
+public class TestSchemaTrackerEarlyReaderSchema extends SubOperatorTest {
+  protected static final CustomErrorContext ERROR_CONTEXT = EmptyErrorContext.INSTANCE;
+  protected static final TupleMetadata SCHEMA = BaseTestSchemaTracker.SCHEMA;
+
+  /**
+   * If a reader column shadows an implicit column, then we treat the
+   * reader column as unprojected and log a warning (that warning is not
+   * tested here.)
+   */
+  @Test
+  public void shadowImplicit() {
+    ProjectionSchemaTracker tracker = trackerFor(
+        RowSetTestUtils.projectList("a",
+            ScanTestUtils.FULLY_QUALIFIED_NAME_COL));
+    ImplicitColumnOptions options = new ImplicitColumnOptions()
+        .optionSet(fixture.getOptionManager());
+    ImplicitColumnResolver parser = new ImplicitColumnResolver(options, ERROR_CONTEXT);
+    parser.parse(tracker);
+
+    TupleMetadata readerSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addNullable(ScanTestUtils.FULLY_QUALIFIED_NAME_COL, MinorType.BIGINT)
+        .build();
+    tracker.applyEarlyReaderSchema(readerSchema);
+
+    TupleMetadata expected = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .add(ScanTestUtils.FULLY_QUALIFIED_NAME_COL, MinorType.VARCHAR)
+        .build();
+    assertEquals(expected, tracker.internalSchema().toSchema());
+  }
+
+  @Test
+  public void testWildcardLenientWithSubset() {
+    ProjectionSchemaTracker tracker = trackerFor(
+        RowSetTestUtils.projectAll());
+    tracker.applyProvidedSchema(SCHEMA);
+    assertTrue(tracker.isResolved());
+    TupleMetadata readerSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .build();
+    tracker.applyEarlyReaderSchema(readerSchema);
+    assertTrue(tracker.isResolved());
+    assertEquals(SCHEMA, tracker.internalSchema().toSchema());
+  }
+
+  @Test
+  public void testWildcardLenientWithSame() {
+    ProjectionSchemaTracker tracker = trackerFor(
+        RowSetTestUtils.projectAll());
+    tracker.applyProvidedSchema(SCHEMA);
+    assertTrue(tracker.isResolved());
+    tracker.applyEarlyReaderSchema(SCHEMA);
+    assertTrue(tracker.isResolved());
+    assertEquals(SCHEMA, tracker.internalSchema().toSchema());
+  }
+
+  @Test
+  public void testWildcardLenientWithSuperset() {
+    ProjectionSchemaTracker tracker = trackerFor(
+        RowSetTestUtils.projectAll());
+    tracker.applyProvidedSchema(SCHEMA);
+    assertTrue(tracker.isResolved());
+    TupleMetadata readerSchema = new SchemaBuilder()
+        .addAll(SCHEMA)
+        .add("c", MinorType.VARCHAR)
+        .build();
+    tracker.applyEarlyReaderSchema(readerSchema);
+    assertTrue(tracker.isResolved());
+    assertEquals(readerSchema, tracker.internalSchema().toSchema());
+  }
+
+  @Test
+  public void testWildcardStrictWithSuperset() {
+    ProjectionSchemaTracker tracker = trackerFor(
+        RowSetTestUtils.projectAll());
+    TupleMetadata provided = SCHEMA.copy();
+    SchemaUtils.markStrict(provided);
+    tracker.applyProvidedSchema(provided);
+    assertTrue(tracker.isResolved());
+
+    TupleMetadata readerSchema = new SchemaBuilder()
+        .addAll(SCHEMA)
+        .add("c", MinorType.VARCHAR)
+        .build();
+    tracker.applyEarlyReaderSchema(readerSchema);
+    assertTrue(tracker.isResolved());
+    assertEquals(SCHEMA, tracker.internalSchema().toSchema());
+  }
+
+  @Test
+  public void testTypeConflict() {
+    ProjectionSchemaTracker tracker = trackerFor(
+        RowSetTestUtils.projectAll());
+    tracker.applyProvidedSchema(SCHEMA);
+    assertTrue(tracker.isResolved());
+    TupleMetadata readerSchema = new SchemaBuilder()
+        .add("a", MinorType.BIGINT)
+        .build();
+    try {
+      tracker.applyEarlyReaderSchema(readerSchema);
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains("type conflict"));
+      assertTrue(e.getMessage().contains("Scan column: `a` INT NOT NULL"));
+      assertTrue(e.getMessage().contains("Reader column: `a` BIGINT NOT NULL"));
+    }
+  }
+
+  @Test
+  public void testModeConflict() {
+    ProjectionSchemaTracker tracker = trackerFor(
+        RowSetTestUtils.projectAll());
+    tracker.applyProvidedSchema(SCHEMA);
+    assertTrue(tracker.isResolved());
+    TupleMetadata readerSchema = new SchemaBuilder()
+        .addNullable("a", MinorType.INT)
+        .build();
+    try {
+      tracker.applyEarlyReaderSchema(readerSchema);
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains("type conflict"));
+      assertTrue(e.getMessage().contains("Scan column: `a` INT NOT NULL"));
+      assertTrue(e.getMessage().contains("Reader column: `a` INT"));
+    }
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/schema/TestSchemaTrackerInputSchema.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/schema/TestSchemaTrackerInputSchema.java
new file mode 100644
index 0000000..40e860f
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/schema/TestSchemaTrackerInputSchema.java
@@ -0,0 +1,609 @@
+/*
+ * 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.v3.schema;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.function.Consumer;
+
+import org.apache.drill.categories.EvfTests;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.rowSet.RowSetTestUtils;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+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 resolving various projection lists against a provided schema.
+ * Things get complex when resolving maps.
+ * <p>
+ * In most cases, an early reader schema, by itself, works the same
+ * as a provided schema: both are a way of providing type information
+ * before any data is seen. There are subtle differences explored
+ * here and in {@link TestSchemaTrackerEarlyReaderSchema}.
+ */
+@Category(EvfTests.class)
+public class TestSchemaTrackerInputSchema extends BaseTestSchemaTracker {
+
+  private void testBoth(Collection<SchemaPath> projList, TupleMetadata schema,
+      Consumer<ProjectionSchemaTracker> test) {
+
+    // Test the schema as provided
+    ProjectionSchemaTracker tracker1 = trackerFor(projList);
+    tracker1.applyProvidedSchema(schema);
+    test.accept(tracker1);
+
+    // Test the schema as an early reader schema
+    ProjectionSchemaTracker tracker2 = trackerFor(projList);
+    tracker2.applyEarlyReaderSchema(schema);
+    test.accept(tracker2);
+  }
+
+  @Test
+  public void testEmpty() {
+    testBoth(RowSetTestUtils.projectNone(), SCHEMA, tracker -> {
+      assertTrue(tracker.isResolved());
+      assertSame(ScanSchemaTracker.ProjectionType.NONE, tracker.projectionType());
+      assertTrue(tracker.internalSchema().toSchema().isEmpty());
+    });
+  }
+
+  @Test
+  public void testWithWildcard() {
+    testBoth(RowSetTestUtils.projectAll(), SCHEMA, tracker -> {
+      assertTrue(tracker.isResolved());
+      assertSame(ScanSchemaTracker.ProjectionType.ALL, tracker.projectionType());
+      TupleMetadata schema = tracker.internalSchema().toSchema();
+      assertEquals(SCHEMA, schema);
+
+      // Verify column properties are merged
+      assertEquals(MOCK_VALUE, schema.metadata("a").property(MOCK_PROP));
+    });
+  }
+
+  @Test
+  public void testStrictWithWildcard() {
+    ProjectionSchemaTracker tracker = trackerFor(
+        RowSetTestUtils.projectAll());
+    TupleMetadata provided = SCHEMA.copy();
+    SchemaUtils.markStrict(provided);
+    tracker.applyProvidedSchema(provided);
+    assertTrue(tracker.isResolved());
+
+    // Strict setting with wildcard only expands the strict
+    // schema; the reader can add no new columns.
+    assertSame(ScanSchemaTracker.ProjectionType.SOME, tracker.projectionType());
+    TupleMetadata schema = tracker.internalSchema().toSchema();
+    assertEquals(SCHEMA, schema);
+  }
+
+  @Test
+  public void testSpecialWithWildcard() {
+    TupleMetadata input = SCHEMA.copy();
+    SchemaUtils.markExcludeFromWildcard(input.metadata("b"));
+    testBoth(RowSetTestUtils.projectAll(), input, tracker -> {
+      assertTrue(tracker.isResolved());
+      assertSame(ScanSchemaTracker.ProjectionType.ALL, tracker.projectionType());
+
+      TupleMetadata expected = new SchemaBuilder()
+          .add("a", MinorType.INT)
+          .build();
+      TupleMetadata schema = tracker.internalSchema().toSchema();
+      assertEquals(expected, schema);
+    });
+  }
+
+  /**
+   * Drill will provide a project list that includes both a wildcard
+   * and column names if the columns are implicit. Not applicable
+   * to a reader schema.
+   */
+  @Test
+  public void testWithWildcardAndCols() {
+    ProjectionSchemaTracker tracker = trackerFor(
+        RowSetTestUtils.projectList("x", SchemaPath.DYNAMIC_STAR, "y"));
+    tracker.applyProvidedSchema(SCHEMA);
+    assertFalse(tracker.isResolved());
+    assertSame(ScanSchemaTracker.ProjectionType.ALL, tracker.projectionType());
+
+    // Schema is partially resolved: some columns are dynamic
+    // (to be resolved by reader)
+    TupleMetadata expected = new SchemaBuilder()
+        .addDynamic("x")
+        .addAll(SCHEMA)
+        .addDynamic("y")
+        .build();
+    assertEquals(expected, tracker.internalSchema().toSchema());
+  }
+
+  @Test
+  public void testWithExplicit() {
+    testBoth(
+        RowSetTestUtils.projectList("b", "c"),
+        SCHEMA, tracker -> {
+      assertSame(ScanSchemaTracker.ProjectionType.SOME, tracker.projectionType());
+      assertFalse(tracker.isResolved());
+      TupleMetadata expected = new SchemaBuilder()
+          .addNullable("b", MinorType.VARCHAR)
+          .addDynamic("c")
+          .build();
+      assertEquals(expected, tracker.internalSchema().toSchema());
+    });
+  }
+
+  @Test
+  public void testWithExplicitReorder() {
+    testBoth(
+        RowSetTestUtils.projectList("b", "a"),
+        SCHEMA, tracker -> {
+      assertSame(ScanSchemaTracker.ProjectionType.SOME, tracker.projectionType());
+      assertTrue(tracker.isResolved());
+      TupleMetadata schema = tracker.internalSchema().toSchema();
+      TupleMetadata expected = new SchemaBuilder()
+          .addNullable("b", MinorType.VARCHAR)
+          .add("a", MinorType.INT)
+          .build();
+      assertEquals(expected, schema);
+
+      // Verify column properties are merged
+      assertEquals(MOCK_VALUE, schema.metadata("a").property(MOCK_PROP));
+    });
+  }
+
+  /**
+   * Verify a reasonable error if the name of an implied implicit
+   * column (one that appears with a wildcard) conflicts with a
+   * provided column name. We cannot project two columns with that
+   * same name. Does not occur for a reader schema.
+   */
+  @Test
+  public void testWithWildcardImplicitConflict() {
+    ProjectionSchemaTracker tracker = trackerFor(
+        RowSetTestUtils.projectList("a", SchemaPath.DYNAMIC_STAR, "y"));
+    try {
+      tracker.applyProvidedSchema(SCHEMA);
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains("implicit"));
+      assertTrue(e.getMessage().contains("Column: a"));
+    }
+  }
+
+  @Test
+  public void testProvidedMapProjectConflict() {
+    ProjectionSchemaTracker tracker = trackerFor(
+        RowSetTestUtils.projectList("a.x"));
+    try {
+      tracker.applyProvidedSchema(SCHEMA);
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains("not compatible"));
+      assertTrue(e.getMessage().contains("Projected column: a {x}"));
+      assertTrue(e.getMessage().contains("Provided column: `a` INT NOT NULL"));
+    }
+  }
+
+  @Test
+  public void testReaderMapProjectConflict() {
+    ProjectionSchemaTracker tracker = trackerFor(
+        RowSetTestUtils.projectList("a.x"));
+    try {
+      tracker.applyEarlyReaderSchema(SCHEMA);
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains("not compatible"));
+      assertTrue(e.getMessage().contains("Projected column: a {x}"));
+      assertTrue(e.getMessage().contains("Reader column: `a` INT NOT NULL"));
+    }
+  }
+
+  @Test
+  public void testProvidedArrayProjectConflict() {
+    ProjectionSchemaTracker tracker = trackerFor(
+        RowSetTestUtils.projectList("a[2]"));
+    try {
+      tracker.applyProvidedSchema(SCHEMA);
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains("not compatible"));
+      assertTrue(e.getMessage().contains("Projected column: a[2]"));
+      assertTrue(e.getMessage().contains("Provided column: `a` INT NOT NULL"));
+    }
+  }
+
+  @Test
+  public void testReaderArrayProjectConflict() {
+    ProjectionSchemaTracker tracker = trackerFor(
+        RowSetTestUtils.projectList("a[2]"));
+    try {
+      tracker.applyEarlyReaderSchema(SCHEMA);
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains("not compatible"));
+      assertTrue(e.getMessage().contains("Projected column: a[2]"));
+      assertTrue(e.getMessage().contains("Reader column: `a` INT NOT NULL"));
+    }
+  }
+
+  @Test
+  public void testEmptyWithMap() {
+    testBoth(
+        Collections.emptyList(),
+        MAP_SCHEMA, tracker -> {
+      assertSame(ScanSchemaTracker.ProjectionType.NONE, tracker.projectionType());
+      assertTrue(tracker.isResolved());
+      assertTrue(tracker.internalSchema().toSchema().isEmpty());
+    });
+  }
+
+  /**
+   * Lenient schema with a map and a wildcard: both the top level
+   * and the map allow the reader to add members.
+   */
+  @Test
+  public void testWithWildcardWithMap() {
+    testBoth(
+        RowSetTestUtils.projectAll(),
+        MAP_SCHEMA, tracker -> {
+      doTestGenericMap(tracker, true);
+      assertSame(ScanSchemaTracker.ProjectionType.ALL, tracker.projectionType());
+    });
+  }
+
+  /**
+   * Strict schema with a map and a wildcard: neither the top level
+   * nor the map allow the reader to add members. Not applicable for
+   * a reader schema.
+   */
+  @Test
+  public void testStrictWithWildcardWithMap() {
+    ProjectionSchemaTracker tracker = trackerFor(
+        RowSetTestUtils.projectAll());
+
+    // Mark schema as strict
+    TupleMetadata provided = MAP_SCHEMA.copy();
+    SchemaUtils.markStrict(provided);
+    tracker.applyProvidedSchema(provided);
+    assertTrue(tracker.isResolved());
+
+    // Projection as without strict, except the map is "closed",
+    // won't accept new members from the reader.
+    TupleMetadata schema = tracker.internalSchema().toSchema();
+    assertEquals(MAP_SCHEMA, schema);
+    ColumnMetadata mapCol = schema.metadata("m");
+    assertFalse(SchemaUtils.isProjectAll(mapCol.tupleSchema()));
+  }
+
+  /**
+   * Test a map projected with just the map name: {@code `m`}.
+   * Expands the the full map.
+   */
+  @Test
+  public void testGenericMap() {
+    testBoth(
+        RowSetTestUtils.projectList("a", "m"),
+        MAP_SCHEMA, tracker -> {
+      doTestGenericMap(tracker, true);
+      assertSame(ScanSchemaTracker.ProjectionType.SOME, tracker.projectionType());
+    });
+  }
+
+  /**
+   * Test a map with members projected in the same order as the
+   * provided schema.
+   */
+  @Test
+  public void testSpecificMap() {
+    testBoth(
+        RowSetTestUtils.projectList("a", "m.x", "m.y"),
+        MAP_SCHEMA, tracker -> {
+      doTestGenericMap(tracker, false);
+      assertSame(ScanSchemaTracker.ProjectionType.SOME, tracker.projectionType());
+    });
+  }
+
+  /**
+   * Projection of whole map and one column, the map
+   * column accepts new reader columns.
+   */
+  @Test
+  public void testGenericAndSpecificMap() {
+    testBoth(
+        RowSetTestUtils.projectList("a", "m.x", "m"),
+        MAP_SCHEMA, tracker -> {
+      doTestGenericMap(tracker, true);
+      assertSame(ScanSchemaTracker.ProjectionType.SOME, tracker.projectionType());
+    });
+  }
+
+  private void doTestGenericMap(ProjectionSchemaTracker tracker, boolean mapAll) {
+    assertTrue(tracker.isResolved());
+    TupleMetadata schema = tracker.internalSchema().toSchema();
+    assertEquals(MAP_SCHEMA, schema);
+    ColumnMetadata mapCol = schema.metadata("m");
+    assertEquals(mapAll, SchemaUtils.isProjectAll(mapCol.tupleSchema()));
+    assertEquals(MOCK_VALUE, schema.metadata("m").property(MOCK_PROP));
+  }
+
+  /**
+   * Lenient schema projecting the whole map: the map allows the
+   * reader to add members.
+   */
+  @Test
+  public void testSubsetWithMap() {
+    TupleMetadata expected = new SchemaBuilder()
+        .addMap("m")
+          .add("x", MinorType.BIGINT)
+          .add("y", MinorType.VARCHAR)
+          .resumeSchema()
+        .build();
+    testBoth(
+        RowSetTestUtils.projectList("m"),
+        MAP_SCHEMA, tracker -> {
+      assertTrue(tracker.isResolved());
+      TupleMetadata schema = tracker.internalSchema().toSchema();
+      assertEquals(expected, schema);
+      assertTrue(SchemaUtils.isProjectAll(
+          schema.metadata("m").tupleSchema()));
+    });
+  }
+
+  /**
+   * Strict schema projecting the whole map: the map does not allow the
+   * reader to add members. Not applicable to a reader schema.
+   */
+  @Test
+  public void testStrictSubsetWithMap() {
+    ProjectionSchemaTracker tracker = trackerFor(
+        RowSetTestUtils.projectList("m"));
+    TupleMetadata provided = MAP_SCHEMA.copy();
+    SchemaUtils.markStrict(provided);
+    tracker.applyProvidedSchema(provided);
+    TupleMetadata expected = new SchemaBuilder()
+        .addMap("m")
+          .add("x", MinorType.BIGINT)
+          .add("y", MinorType.VARCHAR)
+          .resumeSchema()
+        .build();
+    assertTrue(tracker.isResolved());
+    TupleMetadata schema = tracker.internalSchema().toSchema();
+    assertEquals(expected, schema);
+    assertTrue(!SchemaUtils.isProjectAll(
+        schema.metadata("m").tupleSchema()));
+  }
+
+  @Test
+  public void testMapSubset() {
+    TupleMetadata expected = new SchemaBuilder()
+        .addMap("m")
+          .add("x", MinorType.BIGINT)
+          .resumeSchema()
+        .build();
+    testBoth(
+        RowSetTestUtils.projectList("m.x"),
+        MAP_SCHEMA, tracker -> {
+      assertTrue(tracker.isResolved());
+      assertEquals(expected, tracker.internalSchema().toSchema());
+    });
+  }
+
+  @Test
+  public void testMapDisjointSet() {
+    TupleMetadata expected = new SchemaBuilder()
+        .addMap("m")
+          .addDynamic("w")
+          .add("x", MinorType.BIGINT)
+          .addDynamic("z")
+          .resumeSchema()
+        .build();
+    testBoth(
+        RowSetTestUtils.projectList("m.w", "m.x", "m.z"),
+        MAP_SCHEMA, tracker -> {
+      assertFalse(tracker.isResolved());
+      assertEquals(expected, tracker.internalSchema().toSchema());
+    });
+  }
+
+  @Test
+  public void testGenericAndSpecificMapReorder() {
+    TupleMetadata expected = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m")
+          .add("y", MinorType.VARCHAR)
+          .add("x", MinorType.BIGINT)
+          .resumeSchema()
+        .build();
+    testBoth(
+        RowSetTestUtils.projectList("a", "m.y", "m"),
+        MAP_SCHEMA, tracker -> {
+      assertTrue(tracker.isResolved());
+      assertEquals(expected, tracker.internalSchema().toSchema());
+    });
+  }
+
+  @Test
+  public void testProvidedMapProjectConflictInMap() {
+    ProjectionSchemaTracker tracker = trackerFor(
+        RowSetTestUtils.projectList("m.x.u"));
+    try {
+      tracker.applyProvidedSchema(MAP_SCHEMA);
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains("not compatible"));
+      assertTrue(e.getMessage().contains("Projected column: x {u}"));
+      assertTrue(e.getMessage().contains("Provided column: `x` BIGINT NOT NULL"));
+    }
+  }
+
+  @Test
+  public void testReaderMapProjectConflictInMap() {
+    ProjectionSchemaTracker tracker = trackerFor(
+        RowSetTestUtils.projectList("m.x.u"));
+    try {
+      tracker.applyEarlyReaderSchema(MAP_SCHEMA);
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains("not compatible"));
+      assertTrue(e.getMessage().contains("Projected column: x {u}"));
+      assertTrue(e.getMessage().contains("Reader column: `x` BIGINT NOT NULL"));
+    }
+  }
+
+  @Test
+  public void testProvidedArrayProjectConflictInMap() {
+    ProjectionSchemaTracker tracker = trackerFor(
+        RowSetTestUtils.projectList("m.x[2]"));
+    try {
+      tracker.applyProvidedSchema(MAP_SCHEMA);
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains("not compatible"));
+      assertTrue(e.getMessage().contains("Projected column: x[]"));
+      assertTrue(e.getMessage().contains("Provided column: `x` BIGINT NOT NULL"));
+    }
+  }
+
+  @Test
+  public void testReaderArrayProjectConflictInMap() {
+    ProjectionSchemaTracker tracker = trackerFor(
+        RowSetTestUtils.projectList("m.x[2]"));
+    try {
+      tracker.applyEarlyReaderSchema(MAP_SCHEMA);
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains("not compatible"));
+      assertTrue(e.getMessage().contains("Projected column: x[]"));
+      assertTrue(e.getMessage().contains("Reader column: `x` BIGINT NOT NULL"));
+    }
+  }
+
+  @Test
+  public void testWithWildcardWithNestedMap() {
+    testBoth(
+        RowSetTestUtils.projectAll(),
+        NESTED_MAP_SCHEMA, tracker -> {
+      doTestGenericNestedMap(tracker);
+    });
+  }
+
+  @Test
+  public void testGenericNestedMap() {
+    testBoth(
+        RowSetTestUtils.projectList("a", "m"),
+        NESTED_MAP_SCHEMA, tracker -> {
+      doTestGenericNestedMap(tracker);
+    });
+  }
+
+  @Test
+  public void testSpecificNestedMap1() {
+     testBoth(
+        RowSetTestUtils.projectList("a", "m.x", "m.y", "m.m2"),
+        NESTED_MAP_SCHEMA, tracker -> {
+      doTestGenericNestedMap(tracker);
+    });
+  }
+
+  @Test
+  public void testSpecificNestedMap2() {
+    testBoth(
+        RowSetTestUtils.projectList("a", "m.x", "m.y", "m.m2.p", "m.m2.q"),
+        NESTED_MAP_SCHEMA, tracker -> {
+      doTestGenericNestedMap(tracker);
+    });
+  }
+
+  @Test
+  public void testGenericAndSpecificNestedMap() {
+    testBoth(
+        RowSetTestUtils.projectList("a", "m.x", "m.y", "m.m2.p", "m.m2"),
+        NESTED_MAP_SCHEMA, tracker -> {
+      doTestGenericNestedMap(tracker);
+    });
+  }
+
+  private void doTestGenericNestedMap(ProjectionSchemaTracker tracker) {
+    TupleMetadata schema = tracker.internalSchema().toSchema();
+    assertTrue(tracker.isResolved());
+    assertEquals(NESTED_MAP_SCHEMA, schema);
+  }
+
+  @Test
+  public void testSubsetWithNestedMap() {
+    TupleMetadata expected = new SchemaBuilder()
+        .addMap("m")
+          .addMap("m2")
+            .add("p", MinorType.BIGINT)
+            .add("q", MinorType.VARCHAR)
+            .resumeMap()
+          .resumeSchema()
+        .buildSchema();
+    testBoth(
+        RowSetTestUtils.projectList("m.m2"),
+        NESTED_MAP_SCHEMA, tracker -> {
+      assertTrue(tracker.isResolved());
+      assertEquals(expected, tracker.internalSchema().toSchema());
+    });
+  }
+
+  @Test
+  public void testNestedMapSubset() {
+    TupleMetadata expected = new SchemaBuilder()
+        .addMap("m")
+          .addMap("m2")
+            .add("p", MinorType.BIGINT)
+            .resumeMap()
+          .resumeSchema()
+        .buildSchema();
+    testBoth(
+        RowSetTestUtils.projectList("m.m2.p"),
+        NESTED_MAP_SCHEMA, tracker -> {
+      assertTrue(tracker.isResolved());
+      assertEquals(expected, tracker.internalSchema().toSchema());
+    });
+  }
+
+  @Test
+  public void testNestedMapDisjointSet() {
+    TupleMetadata expected = new SchemaBuilder()
+        .addMap("m")
+          .addMap("m2")
+            .addDynamic("o")
+            .add("p", MinorType.BIGINT)
+            .addDynamic("r")
+            .resumeMap()
+          .resumeSchema()
+        .buildSchema();
+    testBoth(
+        RowSetTestUtils.projectList("m.m2.o", "m.m2.p", "m.m2.r"),
+        NESTED_MAP_SCHEMA, tracker -> {
+      assertFalse(tracker.isResolved());
+      assertEquals(expected, tracker.internalSchema().toSchema());
+    });
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/schema/TestSchemaTrackerProjection.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/schema/TestSchemaTrackerProjection.java
new file mode 100644
index 0000000..9587e9c
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/v3/schema/TestSchemaTrackerProjection.java
@@ -0,0 +1,103 @@
+/*
+ * 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.v3.schema;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Collection;
+import java.util.Collections;
+
+import org.apache.drill.categories.EvfTests;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.EmptyErrorContext;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.ScanProjectionParser.ProjectionParseResult;
+import org.apache.drill.exec.physical.resultSet.impl.ProjectionFilter;
+import org.apache.drill.exec.physical.rowSet.RowSetTestUtils;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.test.BaseTest;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test the first step of scan schema resolution: translating from the
+ * projection parser to a dynamic schema ready for resolution.
+ */
+@Category(EvfTests.class)
+public class TestSchemaTrackerProjection extends BaseTest {
+  private static final CustomErrorContext ERROR_CONTEXT = EmptyErrorContext.INSTANCE;
+
+  private ProjectionSchemaTracker schemaTracker(Collection<SchemaPath> projList) {
+    ProjectionParseResult result = ScanProjectionParser.parse(projList);
+    return new ProjectionSchemaTracker(result, true, EmptyErrorContext.INSTANCE);
+  }
+
+  @Test
+  public void testEmpty() {
+    ProjectionSchemaTracker tracker = schemaTracker(
+        Collections.emptyList());
+    assertTrue(tracker.isResolved());
+    assertEquals(0, tracker.schemaVersion());
+    assertSame(ScanSchemaTracker.ProjectionType.NONE, tracker.projectionType());
+    assertTrue(tracker.internalSchema().toSchema().isEmpty());
+    ProjectionFilter filter = tracker.projectionFilter(ERROR_CONTEXT);
+    assertSame(ProjectionFilter.PROJECT_NONE, filter);
+  }
+
+  @Test
+  public void testWildcard() {
+    ProjectionSchemaTracker tracker = schemaTracker(
+        RowSetTestUtils.projectAll());
+    assertFalse(tracker.isResolved());
+    assertEquals(0, tracker.schemaVersion());
+    assertSame(ScanSchemaTracker.ProjectionType.ALL, tracker.projectionType());
+    assertTrue(tracker.internalSchema().toSchema().isEmpty());
+    ProjectionFilter filter = tracker.projectionFilter(ERROR_CONTEXT);
+    assertSame(ProjectionFilter.PROJECT_ALL, filter);
+  }
+
+  @Test
+  public void testWildcardAndCols() {
+    ProjectionSchemaTracker tracker = schemaTracker(
+        RowSetTestUtils.projectList("a", SchemaPath.DYNAMIC_STAR, "b"));
+    assertFalse(tracker.isResolved());
+    assertTrue(0 < tracker.schemaVersion());
+    assertSame(ScanSchemaTracker.ProjectionType.ALL, tracker.projectionType());
+    TupleMetadata schema = tracker.internalSchema().toSchema();
+    assertEquals(2, schema.size());
+    assertTrue(schema.metadata(0).isDynamic());
+    ProjectionFilter filter = tracker.projectionFilter(ERROR_CONTEXT);
+    assertTrue(filter instanceof DynamicSchemaFilter);
+  }
+
+  @Test
+  public void testExplicit() {
+    ProjectionSchemaTracker tracker = schemaTracker(
+        RowSetTestUtils.projectList("a", "b", "c"));
+    assertSame(ScanSchemaTracker.ProjectionType.SOME, tracker.projectionType());
+    assertTrue(0 < tracker.schemaVersion());
+    TupleMetadata schema = tracker.internalSchema().toSchema();
+    assertEquals(3, schema.size());
+    assertTrue(schema.metadata(0).isDynamic());
+    ProjectionFilter filter = tracker.projectionFilter(ERROR_CONTEXT);
+    assertTrue(filter instanceof DynamicSchemaFilter);
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/resultSet/impl/TestProjectionFilter.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/resultSet/impl/TestProjectionFilter.java
new file mode 100644
index 0000000..5c72a45
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/resultSet/impl/TestProjectionFilter.java
@@ -0,0 +1,322 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.resultSet.impl;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.apache.drill.common.exceptions.EmptyErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.physical.resultSet.impl.ProjectionFilter.CompoundProjectionFilter;
+import org.apache.drill.exec.physical.resultSet.impl.ProjectionFilter.DirectProjectionFilter;
+import org.apache.drill.exec.physical.resultSet.impl.ProjectionFilter.ProjResult;
+import org.apache.drill.exec.physical.resultSet.impl.ProjectionFilter.SchemaProjectionFilter;
+import org.apache.drill.exec.physical.resultSet.impl.ProjectionFilter.TypeProjectionFilter;
+import org.apache.drill.exec.physical.resultSet.project.Projections;
+import org.apache.drill.exec.physical.resultSet.project.RequestedTuple;
+import org.apache.drill.exec.physical.rowSet.RowSetTestUtils;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.record.metadata.TupleSchema;
+import org.junit.Test;
+
+public class TestProjectionFilter {
+  private static final ColumnMetadata A_COL = MetadataUtils.newScalar("a", Types.required(MinorType.INT));
+  private static final ColumnMetadata B_COL = MetadataUtils.newScalar("b", Types.optional(MinorType.VARCHAR));
+  private static final ColumnMetadata MAP_COL = MetadataUtils.newMap("m", new TupleSchema());
+  private static final ColumnMetadata MAP_COL2 = MetadataUtils.newMap("m2", new TupleSchema());
+
+  @Test
+  public void testImplicitAll() {
+    ProjectionFilter filter = ProjectionFilter.PROJECT_ALL;
+    assertTrue(filter.isProjected("a"));
+    assertTrue(filter.projection(A_COL).isProjected);
+    ColumnMetadata specialCol = MetadataUtils.newScalar("special", Types.optional(MinorType.BIGINT));
+    specialCol.setBooleanProperty(ColumnMetadata.EXCLUDE_FROM_WILDCARD, true);
+    assertFalse(filter.projection(specialCol).isProjected);
+    assertFalse(filter.isEmpty());
+    ProjResult result = filter.projection(MAP_COL);
+    assertTrue(result.isProjected);
+    assertSame(ProjectionFilter.PROJECT_ALL, result.mapFilter);
+  }
+
+  @Test
+  public void testImplicitNone() {
+    ProjectionFilter filter = ProjectionFilter.PROJECT_NONE;
+    assertFalse(filter.isProjected("a"));
+    assertFalse(filter.projection(A_COL).isProjected);
+    assertTrue(filter.isEmpty());
+    ProjResult result = filter.projection(MAP_COL);
+    assertFalse(result.isProjected);
+    assertSame(ProjectionFilter.PROJECT_NONE, result.mapFilter);
+  }
+
+  @Test
+  public void testProjectList() {
+    RequestedTuple projSet = Projections.parse(
+        RowSetTestUtils.projectList("a", "c", "m.a"));
+    ProjectionFilter filter = new DirectProjectionFilter(projSet, EmptyErrorContext.INSTANCE);
+    assertTrue(filter.isProjected("a"));
+    assertTrue(filter.projection(A_COL).isProjected);
+    assertFalse(filter.isProjected("b"));
+    assertFalse(filter.projection(B_COL).isProjected);
+    assertFalse(filter.isEmpty());
+
+    ProjResult result = filter.projection(MAP_COL);
+    assertTrue(result.isProjected);
+    ProjectionFilter child = result.mapFilter;
+    assertTrue(child.isProjected("a"));
+    assertFalse(child.isProjected("b"));
+
+    result = filter.projection(MAP_COL2);
+    assertFalse(result.isProjected);
+    assertSame(ProjectionFilter.PROJECT_NONE, result.mapFilter);
+  }
+  @Test
+  public void testGenericMap() {
+    RequestedTuple projSet = Projections.parse(
+        RowSetTestUtils.projectList("a", "m"));
+    ProjectionFilter filter = new DirectProjectionFilter(projSet, EmptyErrorContext.INSTANCE);
+    assertTrue(filter.isProjected("a"));
+
+    ProjResult result = filter.projection(MAP_COL);
+    assertTrue(result.isProjected);
+    assertSame(ProjectionFilter.PROJECT_ALL, result.mapFilter);
+  }
+
+  @Test
+  public void testEmptyProjectList() {
+    ProjectionFilter filter = new DirectProjectionFilter(Projections.projectNone(), EmptyErrorContext.INSTANCE);
+    assertFalse(filter.isProjected("a"));
+    assertFalse(filter.projection(A_COL).isProjected);
+    assertTrue(filter.isEmpty());
+    ProjResult result = filter.projection(MAP_COL);
+    assertFalse(result.isProjected);
+    assertSame(ProjectionFilter.PROJECT_NONE, result.mapFilter);
+  }
+
+  @Test
+  public void testTypeFilter() {
+    TupleMetadata schema = new SchemaBuilder()
+        .add(A_COL.copy())
+        .add(B_COL.copy())
+        .addMap("m")
+          .add("a", MinorType.INT)
+          .resumeSchema()
+        .build();
+    ProjectionFilter filter = new TypeProjectionFilter(schema, EmptyErrorContext.INSTANCE);
+    assertFalse(filter.isEmpty());
+    assertTrue(filter.isProjected("a"));
+    assertTrue(filter.projection(A_COL).isProjected);
+    assertTrue(filter.isProjected("b"));
+    assertTrue(filter.projection(B_COL).isProjected);
+    assertTrue(filter.isProjected("c"));
+    assertTrue(filter.projection(
+        MetadataUtils.newScalar("c", Types.required(MinorType.BIGINT))).isProjected);
+
+    ColumnMetadata typeConflict = MetadataUtils.newScalar("a", Types.required(MinorType.BIGINT));
+    try {
+      filter.projection(typeConflict);
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains("conflict"));
+    }
+
+    ColumnMetadata modeConflict = MetadataUtils.newScalar("a", Types.optional(MinorType.INT));
+    try {
+      filter.projection(modeConflict);
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains("conflict"));
+    }
+
+    ProjResult result = filter.projection(MAP_COL);
+    assertTrue(result.isProjected);
+    ProjectionFilter child = result.mapFilter;
+    assertTrue(child.isProjected("a"));
+    assertTrue(child.isProjected("b"));
+
+    result = filter.projection(MAP_COL2);
+    assertTrue(result.isProjected);
+    assertSame(ProjectionFilter.PROJECT_ALL, result.mapFilter);
+
+    try {
+      ColumnMetadata aMap = MetadataUtils.newMap("a", new TupleSchema());
+      filter.projection(aMap);
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains("type conflict"));
+    }
+  }
+
+  @Test
+  public void testSchemaFilter() {
+    TupleMetadata schema = new SchemaBuilder()
+        .add(A_COL.copy())
+        .add(B_COL.copy())
+        .addMap("m")
+          .add("a", MinorType.INT)
+          .resumeSchema()
+        .build();
+    ProjectionFilter filter = new SchemaProjectionFilter(schema, EmptyErrorContext.INSTANCE);
+    assertFalse(filter.isEmpty());
+    assertTrue(filter.isProjected("a"));
+    assertTrue(filter.projection(A_COL).isProjected);
+    assertTrue(filter.isProjected("b"));
+    assertTrue(filter.projection(B_COL).isProjected);
+    assertFalse(filter.isProjected("c"));
+    assertFalse(filter.projection(MetadataUtils.newScalar("c", Types.required(MinorType.BIGINT))).isProjected);
+
+    ColumnMetadata typeConflict = MetadataUtils.newScalar("a", Types.required(MinorType.BIGINT));
+    try {
+      filter.projection(typeConflict);
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains("conflict"));
+    }
+
+    ColumnMetadata modeConflict = MetadataUtils.newScalar("a", Types.optional(MinorType.INT));
+    try {
+      filter.projection(modeConflict);
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains("conflict"));
+    }
+
+    try {
+      ColumnMetadata aMap = MetadataUtils.newMap("a", new TupleSchema());
+      filter.projection(aMap);
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains("type conflict"));
+    }
+
+    ProjResult result = filter.projection(MAP_COL);
+    assertTrue(result.isProjected);
+    ProjectionFilter child = result.mapFilter;
+    assertTrue(child.isProjected("a"));
+    assertFalse(child.isProjected("b"));
+  }
+
+  @Test
+  public void testCompoundFilterMixed1() {
+    ProjectionFilter filter = new CompoundProjectionFilter(
+        ProjectionFilter.PROJECT_ALL, ProjectionFilter.PROJECT_NONE);
+    assertFalse(filter.isProjected("a"));
+    assertFalse(filter.projection(A_COL).isProjected);
+    assertTrue(filter.isEmpty());
+    ProjResult result = filter.projection(MAP_COL);
+    assertFalse(result.isProjected);
+    assertSame(ProjectionFilter.PROJECT_NONE, result.mapFilter);
+  }
+
+  @Test
+  public void testCompoundFilterMixed2() {
+    ProjectionFilter filter = new CompoundProjectionFilter(
+        ProjectionFilter.PROJECT_NONE, ProjectionFilter.PROJECT_ALL);
+    assertFalse(filter.isProjected("a"));
+    assertFalse(filter.projection(A_COL).isProjected);
+    assertTrue(filter.isEmpty());
+    ProjResult result = filter.projection(MAP_COL);
+    assertFalse(result.isProjected);
+    assertSame(ProjectionFilter.PROJECT_NONE, result.mapFilter);
+  }
+
+  @Test
+  public void testCompoundPermissive() {
+    ProjectionFilter filter = new CompoundProjectionFilter(
+        ProjectionFilter.PROJECT_ALL, ProjectionFilter.PROJECT_ALL);
+    assertTrue(filter.isProjected("a"));
+    assertTrue(filter.projection(A_COL).isProjected);
+    ColumnMetadata specialCol = MetadataUtils.newScalar("special", Types.optional(MinorType.BIGINT));
+    specialCol.setBooleanProperty(ColumnMetadata.EXCLUDE_FROM_WILDCARD, true);
+    assertFalse(filter.projection(specialCol).isProjected);
+    assertFalse(filter.isEmpty());
+    ProjResult result = filter.projection(MAP_COL);
+    assertTrue(result.isProjected);
+    assertSame(ProjectionFilter.PROJECT_ALL, result.mapFilter);
+  }
+
+ @Test
+  public void testCompoundMap() {
+    TupleMetadata schema = new SchemaBuilder()
+       .add(A_COL.copy())
+       .add(B_COL.copy())
+       .addMap("m")
+         .add("a", MinorType.INT)
+         .add("b", MinorType.INT)
+         .resumeSchema()
+       .build();
+    ProjectionFilter filter = new CompoundProjectionFilter(
+        new DirectProjectionFilter(Projections.parse(
+            RowSetTestUtils.projectList("a", "c", "m.a")), EmptyErrorContext.INSTANCE),
+        new SchemaProjectionFilter(schema, EmptyErrorContext.INSTANCE));
+
+    ProjResult result = filter.projection(MAP_COL);
+    assertTrue(result.isProjected);
+    assertTrue(result.mapFilter.isProjected("a"));
+ }
+
+ @Test
+ public void testBuilders() {
+    assertSame(ProjectionFilter.PROJECT_ALL,
+        ProjectionFilter.projectionFilter(Projections.projectAll(), EmptyErrorContext.INSTANCE));
+    assertSame(ProjectionFilter.PROJECT_NONE,
+        ProjectionFilter.projectionFilter(Projections.projectNone(), EmptyErrorContext.INSTANCE));
+    assertTrue(
+        ProjectionFilter.projectionFilter(Projections.parse(
+            RowSetTestUtils.projectList("a")), EmptyErrorContext.INSTANCE)
+        instanceof DirectProjectionFilter);
+
+    TupleMetadata schema = new SchemaBuilder()
+        .add(A_COL.copy())
+        .add(B_COL.copy())
+        .build();
+     assertSame(ProjectionFilter.PROJECT_NONE,
+        ProjectionFilter.definedSchemaFilter(new TupleSchema(), EmptyErrorContext.INSTANCE));
+    assertTrue(
+        ProjectionFilter.definedSchemaFilter(schema, EmptyErrorContext.INSTANCE)
+        instanceof SchemaProjectionFilter);
+
+    assertTrue(
+        ProjectionFilter.providedSchemaFilter(Projections.projectAll(), schema,
+            EmptyErrorContext.INSTANCE) instanceof CompoundProjectionFilter);
+    assertSame(ProjectionFilter.PROJECT_NONE,
+        ProjectionFilter.providedSchemaFilter(Projections.projectNone(), schema,
+            EmptyErrorContext.INSTANCE));
+    assertSame(ProjectionFilter.PROJECT_ALL,
+        ProjectionFilter.providedSchemaFilter(Projections.projectAll(), new TupleSchema(),
+            EmptyErrorContext.INSTANCE));
+    TupleMetadata strictEmpty = new TupleSchema();
+    strictEmpty.setBooleanProperty(TupleMetadata.IS_STRICT_SCHEMA_PROP, true);
+    assertSame(ProjectionFilter.PROJECT_NONE,
+        ProjectionFilter.providedSchemaFilter(Projections.projectAll(), strictEmpty,
+            EmptyErrorContext.INSTANCE));
+    assertTrue(
+        ProjectionFilter.providedSchemaFilter(Projections.parse(
+            RowSetTestUtils.projectList("a")),
+            schema, EmptyErrorContext.INSTANCE)
+        instanceof CompoundProjectionFilter);
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/resultSet/impl/TestResultSetLoaderProjection.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/resultSet/impl/TestResultSetLoaderProjection.java
index 1fcb471..1bee697 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/resultSet/impl/TestResultSetLoaderProjection.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/resultSet/impl/TestResultSetLoaderProjection.java
@@ -30,18 +30,22 @@ import static org.junit.Assert.fail;
 import java.util.Arrays;
 import java.util.List;
 
-import org.apache.drill.categories.RowSetTests;
+import org.apache.drill.categories.EvfTests;
 import org.apache.drill.common.exceptions.CustomErrorContext;
 import org.apache.drill.common.exceptions.EmptyErrorContext;
 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.MinorType;
+import org.apache.drill.exec.physical.impl.scan.v3.schema.SchemaUtils;
 import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
 import org.apache.drill.exec.physical.resultSet.RowSetLoader;
 import org.apache.drill.exec.physical.resultSet.impl.ResultSetLoaderImpl.ResultSetOptions;
... 2300 lines suppressed ...