You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by ar...@apache.org on 2018/03/03 18:47:07 UTC

[01/17] drill git commit: DRILL-5994: Enable configuring number of Jetty acceptors and selectors (default to 1 acceptor and 2 selectors)

Repository: drill
Updated Branches:
  refs/heads/master 9073aed67 -> cf2478f7a


DRILL-5994: Enable configuring number of Jetty acceptors and selectors (default to 1 acceptor and 2 selectors)

closes #1148


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/49faae04
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/49faae04
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/49faae04

Branch: refs/heads/master
Commit: 49faae0452935e9ee1054c056df3e038391048ba
Parents: 9073aed
Author: Vlad Rozov <vr...@apache.org>
Authored: Fri Mar 2 10:39:31 2018 -0800
Committer: Arina Ielchiieva <ar...@gmail.com>
Committed: Sat Mar 3 19:47:31 2018 +0200

----------------------------------------------------------------------
 .../org/apache/drill/exec/ExecConstants.java    |  3 +-
 .../drill/exec/server/rest/WebServer.java       | 55 +++++++++-----------
 .../src/main/resources/drill-module.conf        | 11 ++--
 3 files changed, 34 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/49faae04/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
index fb2907d..54fb46a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
@@ -142,6 +142,8 @@ public final class ExecConstants {
   public static final String HTTP_MAX_PROFILES = "drill.exec.http.max_profiles";
   public static final String HTTP_PORT = "drill.exec.http.port";
   public static final String HTTP_PORT_HUNT = "drill.exec.http.porthunt";
+  public static final String HTTP_JETTY_SERVER_ACCEPTORS = "drill.exec.http.jetty.server.acceptors";
+  public static final String HTTP_JETTY_SERVER_SELECTORS = "drill.exec.http.jetty.server.selectors";
   public static final String HTTP_ENABLE_SSL = "drill.exec.http.ssl_enabled";
   public static final String HTTP_CORS_ENABLED = "drill.exec.http.cors.enabled";
   public static final String HTTP_CORS_ALLOWED_ORIGINS = "drill.exec.http.cors.allowedOrigins";
@@ -174,7 +176,6 @@ public final class ExecConstants {
   public static final String USE_LOGIN_PRINCIPAL = "drill.exec.security.bit.auth.use_login_principal";
   public static final String USER_ENCRYPTION_SASL_ENABLED = "drill.exec.security.user.encryption.sasl.enabled";
   public static final String USER_ENCRYPTION_SASL_MAX_WRAPPED_SIZE = "drill.exec.security.user.encryption.sasl.max_wrapped_size";
-  public static final String WEB_SERVER_THREAD_POOL_MAX = "drill.exec.web_server.thread_pool_max";
 
   public static final String USER_SSL_ENABLED = "drill.exec.security.user.encryption.ssl.enabled";
   public static final String BIT_ENCRYPTION_SASL_ENABLED = "drill.exec.security.bit.encryption.sasl.enabled";

http://git-wip-us.apache.org/repos/asf/drill/blob/49faae04/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServer.java
index 4566e7d..bfb6d90 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServer.java
@@ -111,8 +111,6 @@ public class WebServer implements AutoCloseable {
 
   private final Drillbit drillbit;
 
-  private int port;
-
   /**
    * Create Jetty based web server.
    *
@@ -154,35 +152,31 @@ public class WebServer implements AutoCloseable {
 
     final boolean authEnabled = config.getBoolean(ExecConstants.USER_AUTHENTICATION_ENABLED);
 
-    port = config.getInt(ExecConstants.HTTP_PORT);
-    boolean portHunt = config.getBoolean(ExecConstants.HTTP_PORT_HUNT);
-    int retry = 0;
-
-    for (; retry < PORT_HUNT_TRIES; retry++) {
-      embeddedJetty = new Server(new QueuedThreadPool(config.getInt(ExecConstants.WEB_SERVER_THREAD_POOL_MAX)));
-      embeddedJetty.setHandler(createServletContextHandler(authEnabled));
-      embeddedJetty.addConnector(createConnector(port));
-
+    int port = config.getInt(ExecConstants.HTTP_PORT);
+    final boolean portHunt = config.getBoolean(ExecConstants.HTTP_PORT_HUNT);
+    final int acceptors = config.getInt(ExecConstants.HTTP_JETTY_SERVER_ACCEPTORS);
+    final int selectors = config.getInt(ExecConstants.HTTP_JETTY_SERVER_SELECTORS);
+    final QueuedThreadPool threadPool = new QueuedThreadPool(2, 2, 60000);
+    embeddedJetty = new Server(threadPool);
+    embeddedJetty.setHandler(createServletContextHandler(authEnabled));
+    ServerConnector connector = createConnector(port, acceptors, selectors);
+    threadPool.setMaxThreads(1 + connector.getAcceptors() + connector.getSelectorManager().getSelectorCount());
+    embeddedJetty.addConnector(connector);
+    for (int retry = 0; retry < PORT_HUNT_TRIES; retry++) {
+      connector.setPort(port);
       try {
         embeddedJetty.start();
+        return;
       } catch (BindException e) {
         if (portHunt) {
-          int nextPort = port + 1;
-          logger.info("Failed to start on port {}, trying port {}", port, nextPort);
-          port = nextPort;
-          embeddedJetty.stop();
+          logger.info("Failed to start on port {}, trying port {}", port, ++port, e);
           continue;
         } else {
           throw e;
         }
       }
-
-      break;
-    }
-
-    if (retry == PORT_HUNT_TRIES) {
-      throw new IOException("Failed to find a port");
     }
+    throw new IOException("Failed to find a port");
   }
 
   private ServletContextHandler createServletContextHandler(final boolean authEnabled) throws DrillbitStartupException {
@@ -302,23 +296,22 @@ public class WebServer implements AutoCloseable {
   }
 
   public int getPort() {
-    if (!config.getBoolean(ExecConstants.HTTP_ENABLE)) {
+    if (embeddedJetty == null || embeddedJetty.getConnectors().length != 1) {
       throw new UnsupportedOperationException("Http is not enabled");
     }
-
-    return port;
+    return ((ServerConnector)embeddedJetty.getConnectors()[0]).getPort();
   }
 
-  private ServerConnector createConnector(int port) throws Exception {
+  private ServerConnector createConnector(int port, int acceptors, int selectors) throws Exception {
     final ServerConnector serverConnector;
     if (config.getBoolean(ExecConstants.HTTP_ENABLE_SSL)) {
       try {
-        serverConnector = createHttpsConnector(port);
+        serverConnector = createHttpsConnector(port, acceptors, selectors);
       } catch (DrillException e) {
         throw new DrillbitStartupException(e.getMessage(), e);
       }
     } else {
-      serverConnector = createHttpConnector(port);
+      serverConnector = createHttpConnector(port, acceptors, selectors);
     }
 
     return serverConnector;
@@ -331,7 +324,7 @@ public class WebServer implements AutoCloseable {
    * @return Initialized {@link ServerConnector} for HTTPS connections.
    * @throws Exception
    */
-  private ServerConnector createHttpsConnector(int port) throws Exception {
+  private ServerConnector createHttpsConnector(int port, int acceptors, int selectors) throws Exception {
     logger.info("Setting up HTTPS connector for web server");
 
     final SslContextFactory sslContextFactory = new SslContextFactory();
@@ -411,6 +404,7 @@ public class WebServer implements AutoCloseable {
 
     // SSL Connector
     final ServerConnector sslConnector = new ServerConnector(embeddedJetty,
+        null, null, null, acceptors, selectors,
         new SslConnectionFactory(sslContextFactory, HttpVersion.HTTP_1_1.asString()),
         new HttpConnectionFactory(httpsConfig));
     sslConnector.setPort(port);
@@ -424,10 +418,11 @@ public class WebServer implements AutoCloseable {
    * @return Initialized {@link ServerConnector} instance for HTTP connections.
    * @throws Exception
    */
-  private ServerConnector createHttpConnector(int port) throws Exception {
+  private ServerConnector createHttpConnector(int port, int acceptors, int selectors) throws Exception {
     logger.info("Setting up HTTP connector for web server");
     final HttpConfiguration httpConfig = new HttpConfiguration();
-    final ServerConnector httpConnector = new ServerConnector(embeddedJetty, new HttpConnectionFactory(httpConfig));
+    final ServerConnector httpConnector =
+        new ServerConnector(embeddedJetty, null, null, null, acceptors, selectors, new HttpConnectionFactory(httpConfig));
     httpConnector.setPort(port);
 
     return httpConnector;

http://git-wip-us.apache.org/repos/asf/drill/blob/49faae04/exec/java-exec/src/main/resources/drill-module.conf
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/drill-module.conf b/exec/java-exec/src/main/resources/drill-module.conf
index 2305a30..fc365d2 100644
--- a/exec/java-exec/src/main/resources/drill-module.conf
+++ b/exec/java-exec/src/main/resources/drill-module.conf
@@ -120,6 +120,12 @@ drill.exec: {
     ssl_enabled: false,
     porthunt: false,
     port: 8047,
+    jetty : {
+      server : {
+        acceptors : 1,
+        selectors : 2
+      }
+    }
     max_profiles: 100,
     session_max_idle_secs: 3600, # Default value 1hr
     cors: {
@@ -379,10 +385,7 @@ drill.exec: {
   # refresh time.
   grace_period_ms : 0,
   //port hunting for drillbits. Enabled only for testing purposes.
-  port_hunt : false,
-  // Max threads of embedded Jetty
-  web_server.thread_pool_max: 200
-
+  port_hunt : false
 }
 
 drill.jdbc: {


[16/17] drill git commit: DRILL-6114: Metadata revisions

Posted by ar...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/SortTestUtilities.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/SortTestUtilities.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/SortTestUtilities.java
index e106171..093c4ae 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/SortTestUtilities.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/SortTestUtilities.java
@@ -32,7 +32,7 @@ import org.apache.drill.exec.physical.config.Sort;
 import org.apache.drill.exec.physical.impl.xsort.managed.PriorityQueueCopierWrapper.BatchMerger;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
-import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.test.OperatorFixture;
 import org.apache.drill.test.rowSet.DirectRowSet;

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderMapArray.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderMapArray.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderMapArray.java
index 115e52d..86e69aa 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderMapArray.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderMapArray.java
@@ -27,7 +27,7 @@ import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
 import org.apache.drill.exec.physical.rowSet.RowSetLoader;
-import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.accessor.ArrayReader;
 import org.apache.drill.exec.vector.accessor.ArrayWriter;

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderMaps.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderMaps.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderMaps.java
index b23eb0d..2aaa934 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderMaps.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderMaps.java
@@ -29,7 +29,7 @@ import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
 import org.apache.drill.exec.physical.rowSet.RowSetLoader;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
-import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.accessor.ScalarWriter;
 import org.apache.drill.exec.vector.accessor.TupleReader;

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderOmittedValues.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderOmittedValues.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderOmittedValues.java
index 2c4c87b..b0df724 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderOmittedValues.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderOmittedValues.java
@@ -26,7 +26,7 @@ import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
 import org.apache.drill.exec.physical.rowSet.RowSetLoader;
 import org.apache.drill.exec.record.BatchSchema;
-import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.accessor.ScalarWriter;
 import org.apache.drill.test.SubOperatorTest;

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderOverflow.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderOverflow.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderOverflow.java
index 0146cfe..c9ab4e0 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderOverflow.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderOverflow.java
@@ -30,7 +30,7 @@ import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
 import org.apache.drill.exec.physical.rowSet.RowSetLoader;
 import org.apache.drill.exec.physical.rowSet.impl.ResultSetLoaderImpl.ResultSetOptions;
 import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.accessor.ScalarElementReader;
 import org.apache.drill.exec.vector.accessor.ScalarWriter;

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderProjection.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderProjection.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderProjection.java
index 5c6ff7b..e28745b 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderProjection.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderProjection.java
@@ -30,8 +30,8 @@ import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
 import org.apache.drill.exec.physical.rowSet.RowSetLoader;
 import org.apache.drill.exec.physical.rowSet.impl.ResultSetLoaderImpl.ResultSetOptions;
 import org.apache.drill.exec.record.BatchSchema;
-import org.apache.drill.exec.record.ColumnMetadata;
-import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.test.SubOperatorTest;
 import org.apache.drill.test.rowSet.RowSet;

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderProtocol.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderProtocol.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderProtocol.java
index ffcc84a..1c4082b 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderProtocol.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderProtocol.java
@@ -32,7 +32,8 @@ import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
 import org.apache.drill.exec.physical.rowSet.RowSetLoader;
 import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.accessor.ScalarWriter;
 import org.apache.drill.exec.vector.accessor.TupleWriter.UndefinedColumnException;
@@ -98,8 +99,8 @@ public class TestResultSetLoaderProtocol extends SubOperatorTest {
     rootWriter.addColumn(fieldA);
 
     assertEquals(1, schema.size());
-    assertSame(fieldA, schema.column(0));
-    assertSame(fieldA, schema.column("a"));
+    assertTrue(fieldA.isEquivalent(schema.column(0)));
+    assertSame(schema.metadata(0), schema.metadata("a"));
 
     // Error to start a row before the first batch.
 
@@ -148,8 +149,8 @@ public class TestResultSetLoaderProtocol extends SubOperatorTest {
     rootWriter.addColumn(fieldB);
 
     assertEquals(2, schema.size());
-    assertSame(fieldB, schema.column(1));
-    assertSame(fieldB, schema.column("b"));
+    assertTrue(fieldB.isEquivalent(schema.column(1)));
+    assertSame(schema.metadata(1), schema.metadata("b"));
 
     rootWriter.start();
     rootWriter.scalar(0).setInt(200);
@@ -298,6 +299,12 @@ public class TestResultSetLoaderProtocol extends SubOperatorTest {
    * Schemas are case insensitive by default. Verify that
    * the schema mechanism works, with emphasis on the
    * case insensitive case.
+   * <p>
+   * The tests here and elsewhere build columns from a
+   * <tt>MaterializedField</tt>. Doing so is rather old-school;
+   * better to use the newer <tt>ColumnMetadata</tt> which provides
+   * additional information. The code here simply uses the <tt>MaterializedField</tt>
+   * to create a <tt>ColumnMetadata</tt> implicitly.
    */
 
   @Test
@@ -305,10 +312,11 @@ public class TestResultSetLoaderProtocol extends SubOperatorTest {
     ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator());
     RowSetLoader rootWriter = rsLoader.writer();
     TupleMetadata schema = rootWriter.schema();
+    assertEquals(0, rsLoader.schemaVersion());
 
     // No columns defined in schema
 
-    assertNull(schema.column("a"));
+    assertNull(schema.metadata("a"));
     try {
       schema.column(0);
       fail();
@@ -333,14 +341,17 @@ public class TestResultSetLoaderProtocol extends SubOperatorTest {
 
     // Define a column
 
+    assertEquals(0, rsLoader.schemaVersion());
     MaterializedField colSchema = SchemaBuilder.columnSchema("a", MinorType.VARCHAR, DataMode.REQUIRED);
     rootWriter.addColumn(colSchema);
+    assertEquals(1, rsLoader.schemaVersion());
 
     // Can now be found, case insensitive
 
-    assertSame(colSchema, schema.column(0));
-    assertSame(colSchema, schema.column("a"));
-    assertSame(colSchema, schema.column("A"));
+    assertTrue(colSchema.isEquivalent(schema.column(0)));
+    ColumnMetadata colMetadata = schema.metadata(0);
+    assertSame(colMetadata, schema.metadata("a"));
+    assertSame(colMetadata, schema.metadata("A"));
     assertNotNull(rootWriter.column(0));
     assertNotNull(rootWriter.column("a"));
     assertNotNull(rootWriter.column("A"));
@@ -373,9 +384,10 @@ public class TestResultSetLoaderProtocol extends SubOperatorTest {
 
     MaterializedField col2 = SchemaBuilder.columnSchema("b", MinorType.VARCHAR, DataMode.REQUIRED);
     rootWriter.addColumn(col2);
-    assertSame(col2, schema.column(1));
-    assertSame(col2, schema.column("b"));
-    assertSame(col2, schema.column("B"));
+    assertTrue(col2.isEquivalent(schema.column(1)));
+    ColumnMetadata col2Metadata = schema.metadata(1);
+    assertSame(col2Metadata, schema.metadata("b"));
+    assertSame(col2Metadata, schema.metadata("B"));
     assertEquals(2, schema.size());
     assertEquals(1, schema.index("b"));
     assertEquals(1, schema.index("B"));
@@ -391,9 +403,10 @@ public class TestResultSetLoaderProtocol extends SubOperatorTest {
 
     MaterializedField col3 = SchemaBuilder.columnSchema("c", MinorType.VARCHAR, DataMode.REQUIRED);
     rootWriter.addColumn(col3);
-    assertSame(col3, schema.column(2));
-    assertSame(col3, schema.column("c"));
-    assertSame(col3, schema.column("C"));
+    assertTrue(col3.isEquivalent(schema.column(2)));
+    ColumnMetadata col3Metadata = schema.metadata(2);
+    assertSame(col3Metadata, schema.metadata("c"));
+    assertSame(col3Metadata, schema.metadata("C"));
     assertEquals(3, schema.size());
     assertEquals(2, schema.index("c"));
     assertEquals(2, schema.index("C"));
@@ -401,9 +414,10 @@ public class TestResultSetLoaderProtocol extends SubOperatorTest {
 
     MaterializedField col4 = SchemaBuilder.columnSchema("d", MinorType.VARCHAR, DataMode.OPTIONAL);
     rootWriter.addColumn(col4);
-    assertSame(col4, schema.column(3));
-    assertSame(col4, schema.column("d"));
-    assertSame(col4, schema.column("D"));
+    assertTrue(col4.isEquivalent(schema.column(3)));
+    ColumnMetadata col4Metadata = schema.metadata(3);
+    assertSame(col4Metadata, schema.metadata("d"));
+    assertSame(col4Metadata, schema.metadata("D"));
     assertEquals(4, schema.size());
     assertEquals(3, schema.index("d"));
     assertEquals(3, schema.index("D"));
@@ -411,9 +425,10 @@ public class TestResultSetLoaderProtocol extends SubOperatorTest {
 
     MaterializedField col5 = SchemaBuilder.columnSchema("e", MinorType.VARCHAR, DataMode.REPEATED);
     rootWriter.addColumn(col5);
-    assertSame(col5, schema.column(4));
-    assertSame(col5, schema.column("e"));
-    assertSame(col5, schema.column("E"));
+    assertTrue(col5.isEquivalent(schema.column(4)));
+    ColumnMetadata col5Metadata = schema.metadata(4);
+    assertSame(col5Metadata, schema.metadata("e"));
+    assertSame(col5Metadata, schema.metadata("E"));
     assertEquals(5, schema.size());
     assertEquals(4, schema.index("e"));
     assertEquals(4, schema.index("E"));

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderTorture.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderTorture.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderTorture.java
index 33b9826..22a42f6 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderTorture.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderTorture.java
@@ -25,7 +25,7 @@ import java.util.Arrays;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
 import org.apache.drill.exec.physical.rowSet.RowSetLoader;
-import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.accessor.ArrayReader;
 import org.apache.drill.exec.vector.accessor.ArrayWriter;

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestMiniPlan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestMiniPlan.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestMiniPlan.java
index 79b9dbc..e6ddc87 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestMiniPlan.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestMiniPlan.java
@@ -18,10 +18,13 @@
 
 package org.apache.drill.exec.physical.unit;
 
-import com.google.common.collect.Lists;
+import java.util.Collections;
+import java.util.List;
+
 import org.apache.drill.categories.PlannerTest;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.util.DrillFileUtils;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.config.Filter;
 import org.apache.drill.exec.physical.config.UnionAll;
 import org.apache.drill.exec.record.BatchSchema;
@@ -34,8 +37,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import java.util.Collections;
-import java.util.List;
+import com.google.common.collect.Lists;
 
 /**
  * This class contains examples to show how to use MiniPlanTestBuilder to test a
@@ -110,7 +112,7 @@ public class TestMiniPlan extends MiniPlanUnitTestBase {
         "[{\"a\": 50, \"b\" : 100}]");
 
     RecordBatch batch = new PopBuilder()
-        .physicalOperator(new UnionAll(Collections.EMPTY_LIST)) // Children list is provided through RecordBatch
+        .physicalOperator(new UnionAll(Collections.<PhysicalOperator> emptyList())) // Children list is provided through RecordBatch
         .addInputAsChild()
           .physicalOperator(new Filter(null, parseExpr("a=5"), 1.0f))
           .addJsonScanAsChild()

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/test/java/org/apache/drill/exec/record/TestTupleSchema.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/record/TestTupleSchema.java b/exec/java-exec/src/test/java/org/apache/drill/exec/record/TestTupleSchema.java
index 45c0b55..83bdcaf 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/record/TestTupleSchema.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/record/TestTupleSchema.java
@@ -29,12 +29,21 @@ import java.util.Iterator;
 import java.util.List;
 
 import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
-import org.apache.drill.exec.record.TupleSchema.MapColumnMetadata;
-import org.apache.drill.exec.record.TupleSchema.PrimitiveColumnMetadata;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MapColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.PrimitiveColumnMetadata;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.record.metadata.TupleSchema;
+import org.apache.drill.exec.record.metadata.VariantColumnMetadata;
+import org.apache.drill.exec.record.metadata.VariantMetadata;
 import org.apache.drill.test.SubOperatorTest;
 import org.apache.drill.test.rowSet.SchemaBuilder;
+import org.apache.drill.test.rowSet.SchemaBuilder.ColumnBuilder;
 import org.junit.Test;
 
 /**
@@ -53,7 +62,7 @@ public class TestTupleSchema extends SubOperatorTest {
   public void testRequiredFixedWidthColumn() {
 
     MaterializedField field = SchemaBuilder.columnSchema("c", MinorType.INT, DataMode.REQUIRED );
-    ColumnMetadata col = TupleSchema.fromField(field);
+    ColumnMetadata col = MetadataUtils.fromField(field);
 
     // Code may depend on the specific column class
 
@@ -63,31 +72,30 @@ public class TestTupleSchema extends SubOperatorTest {
 
     assertEquals(ColumnMetadata.StructureType.PRIMITIVE, col.structureType());
     assertNull(col.mapSchema());
-    assertSame(field, col.schema());
+    assertTrue(field.isEquivalent(col.schema()));
     assertEquals(field.getName(), col.name());
-    assertEquals(field.getType(), col.majorType());
     assertEquals(field.getType().getMinorType(), col.type());
     assertEquals(field.getDataMode(), col.mode());
     assertFalse(col.isNullable());
     assertFalse(col.isArray());
     assertFalse(col.isVariableWidth());
     assertFalse(col.isMap());
-    assertFalse(col.isList());
     assertTrue(col.isEquivalent(col));
+    assertFalse(col.isVariant());
 
-    ColumnMetadata col2 = TupleSchema.fromField(field);
+    ColumnMetadata col2 = MetadataUtils.fromField(field);
     assertTrue(col.isEquivalent(col2));
 
     MaterializedField field3 = SchemaBuilder.columnSchema("d", MinorType.INT, DataMode.REQUIRED );
-    ColumnMetadata col3 = TupleSchema.fromField(field3);
+    ColumnMetadata col3 = MetadataUtils.fromField(field3);
     assertFalse(col.isEquivalent(col3));
 
     MaterializedField field4 = SchemaBuilder.columnSchema("c", MinorType.BIGINT, DataMode.REQUIRED );
-    ColumnMetadata col4 = TupleSchema.fromField(field4);
+    ColumnMetadata col4 = MetadataUtils.fromField(field4);
     assertFalse(col.isEquivalent(col4));
 
     MaterializedField field5 = SchemaBuilder.columnSchema("c", MinorType.INT, DataMode.OPTIONAL );
-    ColumnMetadata col5 = TupleSchema.fromField(field5);
+    ColumnMetadata col5 = MetadataUtils.fromField(field5);
     assertFalse(col.isEquivalent(col5));
 
     ColumnMetadata col6 = col.cloneEmpty();
@@ -106,14 +114,14 @@ public class TestTupleSchema extends SubOperatorTest {
   public void testNullableFixedWidthColumn() {
 
     MaterializedField field = SchemaBuilder.columnSchema("c", MinorType.INT, DataMode.OPTIONAL );
-    ColumnMetadata col = TupleSchema.fromField(field);
+    ColumnMetadata col = MetadataUtils.fromField(field);
 
     assertEquals(ColumnMetadata.StructureType.PRIMITIVE, col.structureType());
     assertTrue(col.isNullable());
     assertFalse(col.isArray());
     assertFalse(col.isVariableWidth());
     assertFalse(col.isMap());
-    assertFalse(col.isList());
+    assertFalse(col.isVariant());
 
     assertEquals(4, col.expectedWidth());
     col.setExpectedWidth(10);
@@ -128,13 +136,13 @@ public class TestTupleSchema extends SubOperatorTest {
   public void testRepeatedFixedWidthColumn() {
 
     MaterializedField field = SchemaBuilder.columnSchema("c", MinorType.INT, DataMode.REPEATED );
-    ColumnMetadata col = TupleSchema.fromField(field);
+    ColumnMetadata col = MetadataUtils.fromField(field);
 
     assertFalse(col.isNullable());
     assertTrue(col.isArray());
     assertFalse(col.isVariableWidth());
     assertFalse(col.isMap());
-    assertFalse(col.isList());
+    assertFalse(col.isVariant());
 
     assertEquals(4, col.expectedWidth());
     col.setExpectedWidth(10);
@@ -153,7 +161,7 @@ public class TestTupleSchema extends SubOperatorTest {
   public void testRequiredVariableWidthColumn() {
 
     MaterializedField field = SchemaBuilder.columnSchema("c", MinorType.VARCHAR, DataMode.REQUIRED );
-    ColumnMetadata col = TupleSchema.fromField(field);
+    ColumnMetadata col = MetadataUtils.fromField(field);
 
     assertEquals(ColumnMetadata.StructureType.PRIMITIVE, col.structureType());
     assertNull(col.mapSchema());
@@ -161,16 +169,16 @@ public class TestTupleSchema extends SubOperatorTest {
     assertFalse(col.isArray());
     assertTrue(col.isVariableWidth());
     assertFalse(col.isMap());
-    assertFalse(col.isList());
+    assertFalse(col.isVariant());
 
     // A different precision is a different type.
 
-    MaterializedField field2 = new SchemaBuilder.ColumnBuilder("c", MinorType.VARCHAR)
+    MaterializedField field2 = new ColumnBuilder("c", MinorType.VARCHAR)
         .setMode(DataMode.REQUIRED)
         .setPrecision(10)
         .build();
 
-    ColumnMetadata col2 = TupleSchema.fromField(field2);
+    ColumnMetadata col2 = MetadataUtils.fromField(field2);
     assertFalse(col.isEquivalent(col2));
 
     assertEquals(50, col.expectedWidth());
@@ -183,7 +191,7 @@ public class TestTupleSchema extends SubOperatorTest {
 
     // If precision is provided, then that is the default width
 
-    col = TupleSchema.fromField(field2);
+    col = MetadataUtils.fromField(field2);
     assertEquals(10, col.expectedWidth());
   }
 
@@ -191,13 +199,13 @@ public class TestTupleSchema extends SubOperatorTest {
   public void testNullableVariableWidthColumn() {
 
     MaterializedField field = SchemaBuilder.columnSchema("c", MinorType.VARCHAR, DataMode.OPTIONAL );
-    ColumnMetadata col = TupleSchema.fromField(field);
+    ColumnMetadata col = MetadataUtils.fromField(field);
 
     assertTrue(col.isNullable());
     assertFalse(col.isArray());
     assertTrue(col.isVariableWidth());
     assertFalse(col.isMap());
-    assertFalse(col.isList());
+    assertFalse(col.isVariant());
 
     assertEquals(50, col.expectedWidth());
     col.setExpectedWidth(10);
@@ -212,13 +220,13 @@ public class TestTupleSchema extends SubOperatorTest {
   public void testRepeatedVariableWidthColumn() {
 
     MaterializedField field = SchemaBuilder.columnSchema("c", MinorType.VARCHAR, DataMode.REPEATED );
-    ColumnMetadata col = TupleSchema.fromField(field);
+    ColumnMetadata col = MetadataUtils.fromField(field);
 
     assertFalse(col.isNullable());
     assertTrue(col.isArray());
     assertTrue(col.isVariableWidth());
     assertFalse(col.isMap());
-    assertFalse(col.isList());
+    assertFalse(col.isVariant());
 
     assertEquals(50, col.expectedWidth());
     col.setExpectedWidth(10);
@@ -230,6 +238,31 @@ public class TestTupleSchema extends SubOperatorTest {
     assertEquals(2, col.expectedElementCount());
   }
 
+  @Test
+  public void testDecimalScalePrecision() {
+
+    MaterializedField field = MaterializedField.create("d",
+        MajorType.newBuilder()
+          .setMinorType(MinorType.DECIMAL9)
+          .setMode(DataMode.REQUIRED)
+          .setPrecision(3)
+          .setScale(4)
+          .build());
+
+    ColumnMetadata col = MetadataUtils.fromField(field);
+
+    assertFalse(col.isNullable());
+    assertFalse(col.isArray());
+    assertFalse(col.isVariableWidth());
+    assertFalse(col.isMap());
+    assertFalse(col.isVariant());
+
+    assertEquals(3, col.precision());
+    assertEquals(4, col.scale());
+
+    assertTrue(field.isEquivalent(col.schema()));
+  }
+
   /**
    * Tests a map column. Maps can only be required or repeated, not nullable.
    * (But, the columns in the map can be nullable.)
@@ -239,7 +272,7 @@ public class TestTupleSchema extends SubOperatorTest {
   public void testMapColumn() {
 
     MaterializedField field = SchemaBuilder.columnSchema("m", MinorType.MAP, DataMode.REQUIRED );
-    ColumnMetadata col = TupleSchema.fromField(field);
+    ColumnMetadata col = MetadataUtils.fromField(field);
 
     assertTrue(col instanceof MapColumnMetadata);
     assertNotNull(col.mapSchema());
@@ -254,7 +287,7 @@ public class TestTupleSchema extends SubOperatorTest {
     assertFalse(col.isArray());
     assertFalse(col.isVariableWidth());
     assertTrue(col.isMap());
-    assertFalse(col.isList());
+    assertFalse(col.isVariant());
 
     assertEquals(0, col.expectedWidth());
     col.setExpectedWidth(10);
@@ -269,7 +302,7 @@ public class TestTupleSchema extends SubOperatorTest {
   public void testRepeatedMapColumn() {
 
     MaterializedField field = SchemaBuilder.columnSchema("m", MinorType.MAP, DataMode.REPEATED );
-    ColumnMetadata col = TupleSchema.fromField(field);
+    ColumnMetadata col = MetadataUtils.fromField(field);
 
     assertTrue(col instanceof MapColumnMetadata);
     assertNotNull(col.mapSchema());
@@ -279,7 +312,7 @@ public class TestTupleSchema extends SubOperatorTest {
     assertTrue(col.isArray());
     assertFalse(col.isVariableWidth());
     assertTrue(col.isMap());
-    assertFalse(col.isList());
+    assertFalse(col.isVariant());
 
     assertEquals(0, col.expectedWidth());
     col.setExpectedWidth(10);
@@ -291,9 +324,75 @@ public class TestTupleSchema extends SubOperatorTest {
     assertEquals(2, col.expectedElementCount());
   }
 
-    // List
+  @Test
+  public void testUnionColumn() {
+
+    MaterializedField field = SchemaBuilder.columnSchema("u", MinorType.UNION, DataMode.OPTIONAL);
+    ColumnMetadata col = MetadataUtils.fromField(field);
+    assertFalse(col.isArray());
+    doVariantTest(col);
+  }
+
+  @Test
+  public void testListColumn() {
+
+    MaterializedField field = SchemaBuilder.columnSchema("l", MinorType.LIST, DataMode.OPTIONAL);
+    ColumnMetadata col = MetadataUtils.fromField(field);
+    assertTrue(col.isArray());
+
+    // List modeled as a repeated element. Implementation is a bit
+    // more complex, but does not affect this abstract description.
+
+    doVariantTest(col);
+  }
+
+  private void doVariantTest(ColumnMetadata col) {
 
-    // Repeated list
+    assertTrue(col instanceof VariantColumnMetadata);
+
+    assertTrue(col.isNullable());
+    assertFalse(col.isVariableWidth());
+    assertFalse(col.isMap());
+    assertTrue(col.isVariant());
+
+    assertEquals(0, col.expectedWidth());
+    col.setExpectedWidth(10);
+    assertEquals(0, col.expectedWidth());
+
+    VariantMetadata variant = col.variantSchema();
+    assertNotNull(variant);
+    assertEquals(0, variant.size());
+
+    ColumnMetadata member = variant.addType(MinorType.INT);
+    assertEquals(MinorType.INT, member.type());
+    assertEquals(DataMode.OPTIONAL, member.mode());
+    assertEquals(Types.typeKey(MinorType.INT), member.name());
+
+    assertEquals(1, variant.size());
+    assertTrue(variant.hasType(MinorType.INT));
+    assertSame(member, variant.member(MinorType.INT));
+
+    try {
+      variant.addType(MinorType.INT);
+      fail();
+    } catch (IllegalArgumentException e) {
+      // expected
+    }
+
+    assertFalse(variant.hasType(MinorType.VARCHAR));
+    member = variant.addType(MinorType.VARCHAR);
+    assertEquals(MinorType.VARCHAR, member.type());
+    assertEquals(DataMode.OPTIONAL, member.mode());
+    assertEquals(Types.typeKey(MinorType.VARCHAR), member.name());
+
+    assertEquals(2, variant.size());
+    assertTrue(variant.hasType(MinorType.VARCHAR));
+    assertSame(member, variant.member(MinorType.VARCHAR));
+
+    assertFalse(variant.hasType(MinorType.BIGINT));
+  }
+
+  // Repeated list
 
   /**
    * Test the basics of an empty root tuple (i.e. row) schema.
@@ -335,7 +434,7 @@ public class TestTupleSchema extends SubOperatorTest {
     // in the tuple.
 
     MaterializedField field = SchemaBuilder.columnSchema("c", MinorType.INT, DataMode.REQUIRED );
-    ColumnMetadata col = TupleSchema.fromField(field);
+    ColumnMetadata col = MetadataUtils.fromField(field);
     assertEquals("c", root.fullName(col));
 
     assertTrue(root.isEquivalent(root));
@@ -361,9 +460,9 @@ public class TestTupleSchema extends SubOperatorTest {
     assertEquals(0, root.index("a"));
     assertEquals(-1, root.index("b"));
 
-    assertSame(fieldA, root.column(0));
-    assertSame(fieldA, root.column("a"));
-    assertSame(fieldA, root.column("A"));
+    assertTrue(fieldA.isEquivalent(root.column(0)));
+    assertTrue(fieldA.isEquivalent(root.column("a")));
+    assertTrue(fieldA.isEquivalent(root.column("A")));
 
     assertSame(colA, root.metadata(0));
     assertSame(colA, root.metadata("a"));
@@ -379,7 +478,7 @@ public class TestTupleSchema extends SubOperatorTest {
     }
 
     MaterializedField fieldB = SchemaBuilder.columnSchema("b", MinorType.VARCHAR, DataMode.OPTIONAL );
-    ColumnMetadata colB = TupleSchema.fromField(fieldB);
+    ColumnMetadata colB = MetadataUtils.fromField(fieldB);
     int indexB = root.addColumn(colB);
 
     assertEquals(1, indexB);
@@ -387,8 +486,8 @@ public class TestTupleSchema extends SubOperatorTest {
     assertFalse(root.isEmpty());
     assertEquals(indexB, root.index("b"));
 
-    assertSame(fieldB, root.column(1));
-    assertSame(fieldB, root.column("b"));
+    assertTrue(fieldB.isEquivalent(root.column(1)));
+    assertTrue(fieldB.isEquivalent(root.column("b")));
 
     assertSame(colB, root.metadata(1));
     assertSame(colB, root.metadata("b"));
@@ -404,8 +503,8 @@ public class TestTupleSchema extends SubOperatorTest {
     }
 
     List<MaterializedField> fieldList = root.toFieldList();
-    assertSame(fieldA, fieldList.get(0));
-    assertSame(fieldB, fieldList.get(1));
+    assertTrue(fieldA.isEquivalent(fieldList.get(0)));
+    assertTrue(fieldB.isEquivalent(fieldList.get(1)));
 
     TupleMetadata emptyRoot = new TupleSchema();
     assertFalse(emptyRoot.isEquivalent(root));
@@ -433,7 +532,7 @@ public class TestTupleSchema extends SubOperatorTest {
     // And it is equivalent to the round trip to a batch schema.
 
     BatchSchema batchSchema = ((TupleSchema) root).toBatchSchema(SelectionVectorMode.NONE);
-    assertTrue(root.isEquivalent(TupleSchema.fromFields(batchSchema)));
+    assertTrue(root.isEquivalent(MetadataUtils.fromFields(batchSchema)));
   }
 
   /**
@@ -441,10 +540,12 @@ public class TestTupleSchema extends SubOperatorTest {
    * a.`b.x`.`c.y`.d<br>
    * in which columns "a", "b.x" and "c.y" are maps, "b.x" and "c.y" are names
    * that contains dots, and d is primitive.
+   * Here we build up the schema using the metadata schema, and generate a
+   * materialized field from the metadata.
    */
 
   @Test
-  public void testMapTuple() {
+  public void testMapTupleFromMetadata() {
 
     TupleMetadata root = new TupleSchema();
 
@@ -492,18 +593,91 @@ public class TestTupleSchema extends SubOperatorTest {
 
     // Yes, it is awful that MaterializedField does not provide indexed
     // access to its children. That's one reason we have the TupleMetadata
-    // classes..
+    // classes...
+    // Note that the metadata layer does not store the materialized field.
+    // (Doing so causes no end of synchronization problems.) So we test
+    // for equivalence, not sameness.
 
-    assertSame(fieldB, colA.schema().getChildren().iterator().next());
-    assertSame(fieldC, colB.schema().getChildren().iterator().next());
     Iterator<MaterializedField> iterC = colC.schema().getChildren().iterator();
-    assertSame(fieldD, iterC.next());
-    assertSame(fieldE, iterC.next());
+    assertTrue(fieldD.isEquivalent(iterC.next()));
+    assertTrue(fieldE.isEquivalent(iterC.next()));
 
     // Copying should be deep.
 
     TupleMetadata root2 = ((TupleSchema) root).copy();
     assertEquals(2, root2.metadata(0).mapSchema().metadata(0).mapSchema().metadata(0).mapSchema().size());
     assert(root.isEquivalent(root2));
+
+    // Generate a materialized field and compare.
+
+    fieldA.addChild(fieldB);
+    fieldB.addChild(fieldC);
+    fieldC.addChild(fieldD);
+    fieldC.addChild(fieldE);
+    assertTrue(colA.schema().isEquivalent(fieldA));
+  }
+
+  @Test
+  public void testMapTupleFromField() {
+
+    // Create a materialized field with the desired structure.
+
+    MaterializedField fieldA = SchemaBuilder.columnSchema("a", MinorType.MAP, DataMode.REQUIRED);
+
+    MaterializedField fieldB = SchemaBuilder.columnSchema("b.x", MinorType.MAP, DataMode.REQUIRED);
+    fieldA.addChild(fieldB);
+
+    MaterializedField fieldC = SchemaBuilder.columnSchema("c.y", MinorType.MAP, DataMode.REQUIRED);
+    fieldB.addChild(fieldC);
+
+    MaterializedField fieldD = SchemaBuilder.columnSchema("d", MinorType.VARCHAR, DataMode.REQUIRED);
+    fieldC.addChild(fieldD);
+
+    MaterializedField fieldE = SchemaBuilder.columnSchema("e", MinorType.INT, DataMode.REQUIRED);
+    fieldC.addChild(fieldE);
+
+    // Create a metadata schema from the field.
+
+    TupleMetadata root = new TupleSchema();
+    ColumnMetadata colA = root.add(fieldA);
+
+    // Get the parts.
+
+    TupleMetadata mapA = colA.mapSchema();
+    ColumnMetadata colB = mapA.metadata("b.x");
+    TupleMetadata mapB = colB.mapSchema();
+    ColumnMetadata colC = mapB.metadata("c.y");
+    TupleMetadata mapC = colC.mapSchema();
+    ColumnMetadata colD = mapC.metadata("d");
+    ColumnMetadata colE = mapC.metadata("e");
+
+    // Validate. Should be same as earlier test that started
+    // with the metadata.
+
+    assertEquals(1, root.size());
+    assertEquals(1, mapA.size());
+    assertEquals(1, mapB.size());
+    assertEquals(2, mapC.size());
+
+    assertSame(colA, root.metadata("a"));
+    assertSame(colB, mapA.metadata("b.x"));
+    assertSame(colC, mapB.metadata("c.y"));
+    assertSame(colD, mapC.metadata("d"));
+    assertSame(colE, mapC.metadata("e"));
+
+    // The full name contains quoted names if the contain dots.
+    // This name is more for diagnostic than semantic purposes.
+
+    assertEquals("a", root.fullName(0));
+    assertEquals("a.`b.x`", mapA.fullName(0));
+    assertEquals("a.`b.x`.`c.y`", mapB.fullName(0));
+    assertEquals("a.`b.x`.`c.y`.d", mapC.fullName(0));
+    assertEquals("a.`b.x`.`c.y`.e", mapC.fullName(1));
+
+    assertEquals(1, colA.schema().getChildren().size());
+    assertEquals(1, colB.schema().getChildren().size());
+    assertEquals(2, colC.schema().getChildren().size());
+
+    assertTrue(colA.schema().isEquivalent(fieldA));
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/text/compliant/TestCsv.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/text/compliant/TestCsv.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/text/compliant/TestCsv.java
index f0cc172..5b055af 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/text/compliant/TestCsv.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/text/compliant/TestCsv.java
@@ -17,7 +17,6 @@
  */
 package org.apache.drill.exec.store.easy.text.compliant;
 
-import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -142,7 +141,7 @@ public class TestCsv extends ClusterTest {
         .add("b", MinorType.VARCHAR)
         .add("C", MinorType.VARCHAR)
         .build();
-    assertEquals(expectedSchema, actual.batchSchema());
+    assertTrue(expectedSchema.isEquivalent(actual.batchSchema()));
 
     RowSet expected = new RowSetBuilder(client.allocator(), expectedSchema)
         .addRow("10", "foo", "bar")

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/test/java/org/apache/drill/test/DrillTestWrapper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/DrillTestWrapper.java b/exec/java-exec/src/test/java/org/apache/drill/test/DrillTestWrapper.java
index 78e32ee..b5450e6 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/DrillTestWrapper.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/DrillTestWrapper.java
@@ -350,7 +350,7 @@ public class DrillTestWrapper {
     for (VectorAccessible loader : batches)  {
       numBatch++;
       if (expectedSchema != null) {
-        if (! expectedSchema.equals(loader.getSchema())) {
+        if (! expectedSchema.isEquivalent(loader.getSchema())) {
           throw new SchemaChangeException(String.format("Batch schema does not match expected schema\n" +
                   "Actual schema: %s.  Expected schema : %s",
               loader.getSchema(), expectedSchema));
@@ -469,8 +469,8 @@ public class DrillTestWrapper {
         final String expectedSchemaPath = expectedSchema.get(i).getLeft().getRootSegmentPath();
         final TypeProtos.MajorType expectedMajorType = expectedSchema.get(i).getValue();
 
-        if (!actualSchemaPath.equals(expectedSchemaPath)
-            || !actualMajorType.equals(expectedMajorType)) {
+        if (! actualSchemaPath.equals(expectedSchemaPath) ||
+            ! Types.isEquivalent(actualMajorType, expectedMajorType)) {
           throw new Exception(String.format("Schema path or type mismatch for column #%d:\n" +
                   "Expected schema path: %s\nActual   schema path: %s\nExpected type: %s\nActual   type: %s",
               i, expectedSchemaPath, actualSchemaPath, Types.toString(expectedMajorType),

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/test/java/org/apache/drill/test/OperatorFixture.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/OperatorFixture.java b/exec/java-exec/src/test/java/org/apache/drill/test/OperatorFixture.java
index 2cd5d02..3e50f75 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/OperatorFixture.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/OperatorFixture.java
@@ -18,15 +18,10 @@
 package org.apache.drill.test;
 
 import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 
-import com.google.common.base.Function;
-import com.google.common.base.Preconditions;
-import io.netty.buffer.DrillBuf;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.scanner.ClassPathScanner;
@@ -45,19 +40,17 @@ import org.apache.drill.exec.ops.BufferManager;
 import org.apache.drill.exec.ops.BufferManagerImpl;
 import org.apache.drill.exec.ops.ContextInformation;
 import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.ops.MetricDef;
 import org.apache.drill.exec.ops.OpProfileDef;
 import org.apache.drill.exec.ops.OperatorContext;
-import org.apache.drill.exec.ops.OperatorStatReceiver;
 import org.apache.drill.exec.ops.OperatorStats;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.planner.PhysicalPlanReaderTestFactory;
 import org.apache.drill.exec.proto.ExecProtos;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
-import org.apache.drill.exec.record.TupleMetadata;
-import org.apache.drill.exec.record.TupleSchema;
 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.record.selection.SelectionVector2;
 import org.apache.drill.exec.server.options.OptionManager;
 import org.apache.drill.exec.server.options.SystemOptionManager;
@@ -73,8 +66,12 @@ import org.apache.drill.test.rowSet.RowSet.ExtendableRowSet;
 import org.apache.drill.test.rowSet.RowSetBuilder;
 import org.apache.hadoop.security.UserGroupInformation;
 
+import com.google.common.base.Function;
+import com.google.common.base.Preconditions;
 import com.google.common.util.concurrent.ListenableFuture;
 
+import io.netty.buffer.DrillBuf;
+
 /**
  * Test fixture for operator and (especially) "sub-operator" tests.
  * These are tests that are done without the full Drillbit server.
@@ -375,7 +372,7 @@ public class OperatorFixture extends BaseFixture implements AutoCloseable {
   }
 
   public RowSetBuilder rowSetBuilder(BatchSchema schema) {
-    return rowSetBuilder(TupleSchema.fromFields(schema));
+    return rowSetBuilder(MetadataUtils.fromFields(schema));
   }
 
   public RowSetBuilder rowSetBuilder(TupleMetadata schema) {

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/AbstractRowSet.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/AbstractRowSet.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/AbstractRowSet.java
index d128e4f..2c8491b 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/AbstractRowSet.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/AbstractRowSet.java
@@ -19,9 +19,9 @@ package org.apache.drill.test.rowSet;
 
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.record.BatchSchema;
-import org.apache.drill.exec.record.TupleMetadata;
 import org.apache.drill.exec.record.VectorAccessible;
 import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.vector.SchemaChangeCallBack;
 
 /**

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/AbstractSingleRowSet.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/AbstractSingleRowSet.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/AbstractSingleRowSet.java
index ef41b3a..71ca3cf 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/AbstractSingleRowSet.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/AbstractSingleRowSet.java
@@ -21,8 +21,8 @@ import org.apache.drill.exec.record.RecordBatchSizer;
 import org.apache.drill.exec.physical.rowSet.model.ReaderIndex;
 import org.apache.drill.exec.physical.rowSet.model.MetadataProvider.MetadataRetrieval;
 import org.apache.drill.exec.physical.rowSet.model.single.BaseReaderBuilder;
-import org.apache.drill.exec.record.TupleMetadata;
 import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
 
 /**

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/DirectRowSet.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/DirectRowSet.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/DirectRowSet.java
index b5b1f1f..9262706 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/DirectRowSet.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/DirectRowSet.java
@@ -19,18 +19,18 @@ package org.apache.drill.test.rowSet;
 
 import com.google.common.collect.Sets;
 import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.physical.rowSet.model.MetadataProvider.MetadataRetrieval;
 import org.apache.drill.exec.physical.rowSet.model.ReaderIndex;
 import org.apache.drill.exec.physical.rowSet.model.SchemaInference;
-import org.apache.drill.exec.physical.rowSet.model.MetadataProvider.MetadataRetrieval;
 import org.apache.drill.exec.physical.rowSet.model.single.BaseWriterBuilder;
 import org.apache.drill.exec.physical.rowSet.model.single.BuildVectorsFromMetadata;
 import org.apache.drill.exec.physical.rowSet.model.single.VectorAllocator;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
-import org.apache.drill.exec.record.TupleMetadata;
-import org.apache.drill.exec.record.TupleSchema;
 import org.apache.drill.exec.record.VectorAccessible;
 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.record.selection.SelectionVector2;
 import org.apache.drill.test.rowSet.RowSet.ExtendableRowSet;
 import org.apache.drill.test.rowSet.RowSetWriterImpl.WriterIndexImpl;
@@ -85,7 +85,7 @@ public class DirectRowSet extends AbstractSingleRowSet implements ExtendableRowS
   }
 
   public static DirectRowSet fromSchema(BufferAllocator allocator, BatchSchema schema) {
-    return fromSchema(allocator, TupleSchema.fromFields(schema));
+    return fromSchema(allocator, MetadataUtils.fromFields(schema));
   }
 
   public static DirectRowSet fromSchema(BufferAllocator allocator, TupleMetadata schema) {

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/HyperRowSetImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/HyperRowSetImpl.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/HyperRowSetImpl.java
index d0ca662..70b8a20 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/HyperRowSetImpl.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/HyperRowSetImpl.java
@@ -21,7 +21,7 @@ import org.apache.drill.exec.physical.rowSet.model.MetadataProvider.MetadataRetr
 import org.apache.drill.exec.physical.rowSet.model.SchemaInference;
 import org.apache.drill.exec.physical.rowSet.model.hyper.BaseReaderBuilder;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
-import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.selection.SelectionVector4;
 import org.apache.drill.test.rowSet.RowSet.HyperRowSet;

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSet.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSet.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSet.java
index ec0925e..53be75d 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSet.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSet.java
@@ -20,7 +20,7 @@ package org.apache.drill.test.rowSet;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
-import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.record.VectorAccessible;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.selection.SelectionVector2;

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetBuilder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetBuilder.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetBuilder.java
index d6bbaf8..c1a98ac 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetBuilder.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetBuilder.java
@@ -20,8 +20,8 @@ package org.apache.drill.test.rowSet;
 import com.google.common.collect.Sets;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.record.BatchSchema;
-import org.apache.drill.exec.record.TupleMetadata;
-import org.apache.drill.exec.record.TupleSchema;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.vector.accessor.TupleWriter;
 import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
 
@@ -46,7 +46,7 @@ public final class RowSetBuilder {
   private Set<Integer> skipIndices = Sets.newHashSet();
 
   public RowSetBuilder(BufferAllocator allocator, BatchSchema schema) {
-    this(allocator, TupleSchema.fromFields(schema), 10);
+    this(allocator, MetadataUtils.fromFields(schema), 10);
   }
 
   public RowSetBuilder(BufferAllocator allocator, TupleMetadata schema) {

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetPrinter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetPrinter.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetPrinter.java
index e730987..fe50197 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetPrinter.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetPrinter.java
@@ -20,8 +20,8 @@ package org.apache.drill.test.rowSet;
 import java.io.PrintStream;
 
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
-import org.apache.drill.exec.record.ColumnMetadata;
-import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 
 /**
  * Print a row set in CSV-like format. Primarily for debugging.

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetReaderImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetReaderImpl.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetReaderImpl.java
index 2bae085..7217187 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetReaderImpl.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetReaderImpl.java
@@ -20,7 +20,7 @@ package org.apache.drill.test.rowSet;
 import java.util.List;
 
 import org.apache.drill.exec.physical.rowSet.model.ReaderIndex;
-import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.vector.accessor.reader.AbstractObjectReader;
 import org.apache.drill.exec.vector.accessor.reader.AbstractTupleReader;
 

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetWriterImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetWriterImpl.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetWriterImpl.java
index 074842d..b649b11 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetWriterImpl.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetWriterImpl.java
@@ -19,7 +19,7 @@ package org.apache.drill.test.rowSet;
 
 import java.util.List;
 
-import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.accessor.ColumnWriterIndex;
 import org.apache.drill.exec.vector.accessor.writer.AbstractObjectWriter;

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/SchemaBuilder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/SchemaBuilder.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/SchemaBuilder.java
index 9223ef4..3d4df05 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/SchemaBuilder.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/SchemaBuilder.java
@@ -22,11 +22,12 @@ import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
-import org.apache.drill.exec.record.ColumnMetadata;
 import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.record.TupleMetadata;
-import org.apache.drill.exec.record.TupleSchema;
-import org.apache.drill.exec.record.TupleSchema.MapColumnMetadata;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MapColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.record.metadata.TupleSchema;
 
 /**
  * Builder of a row set schema expressed as a list of materialized
@@ -128,7 +129,7 @@ public class SchemaBuilder {
       for (ColumnMetadata md : schema) {
         col.addChild(md.schema());
       }
-      parent.finishMap(TupleSchema.newMap(col, schema));
+      parent.finishMap(MetadataUtils.newMap(col, schema));
       return parent;
     }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/DummyWriterTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/DummyWriterTest.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/DummyWriterTest.java
index db33b30..46f1cc3 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/DummyWriterTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/DummyWriterTest.java
@@ -23,7 +23,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.drill.common.types.TypeProtos.MinorType;
-import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.vector.accessor.writer.AbstractObjectWriter;
 import org.apache.drill.exec.vector.accessor.writer.AbstractTupleWriter;
 import org.apache.drill.exec.vector.accessor.writer.ColumnWriterFactory;

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/PerformanceTool.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/PerformanceTool.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/PerformanceTool.java
index ca282a1..4819253 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/PerformanceTool.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/PerformanceTool.java
@@ -21,10 +21,10 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.common.types.TypeProtos.MinorType;
-import org.apache.drill.exec.record.ColumnMetadata;
 import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.record.TupleMetadata;
-import org.apache.drill.exec.record.TupleSchema;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.vector.IntVector;
 import org.apache.drill.exec.vector.NullableIntVector;
 import org.apache.drill.exec.vector.RepeatedIntVector;
@@ -248,7 +248,7 @@ public class PerformanceTool {
       try (RepeatedIntVector vector = new RepeatedIntVector(rowSchema.column(0), fixture.allocator());) {
         vector.allocateNew(ROW_COUNT, 5 * ROW_COUNT);
         IntColumnWriter colWriter = new IntColumnWriter(vector.getDataVector());
-        ColumnMetadata colSchema = TupleSchema.fromField(vector.getField());
+        ColumnMetadata colSchema = MetadataUtils.fromField(vector.getField());
         ArrayObjectWriter arrayWriter = ScalarArrayWriter.build(colSchema, vector, colWriter);
         TestWriterIndex index = new TestWriterIndex();
         arrayWriter.events().bindIndex(index);

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/RowSetTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/RowSetTest.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/RowSetTest.java
index 4db4d09..5ba1c54 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/RowSetTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/RowSetTest.java
@@ -28,7 +28,7 @@ import java.util.Arrays;
 
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.record.BatchSchema;
-import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.accessor.ArrayReader;
 import org.apache.drill.exec.vector.accessor.ArrayWriter;

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/TestFillEmpties.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/TestFillEmpties.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/TestFillEmpties.java
index 147b713..f9f1b8a 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/TestFillEmpties.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/TestFillEmpties.java
@@ -23,7 +23,7 @@ import static org.junit.Assert.assertTrue;
 import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.TypeProtos.MinorType;
-import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.vector.accessor.ScalarElementReader;
 import org.apache.drill.exec.vector.accessor.ScalarReader;
 import org.apache.drill.exec.vector.accessor.ScalarWriter;

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/vector/src/main/java/org/apache/drill/exec/record/ColumnMetadata.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/record/ColumnMetadata.java b/exec/vector/src/main/java/org/apache/drill/exec/record/ColumnMetadata.java
deleted file mode 100644
index 558aab8..0000000
--- a/exec/vector/src/main/java/org/apache/drill/exec/record/ColumnMetadata.java
+++ /dev/null
@@ -1,114 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.record;
-
-import org.apache.drill.common.types.TypeProtos.DataMode;
-import org.apache.drill.common.types.TypeProtos.MajorType;
-import org.apache.drill.common.types.TypeProtos.MinorType;
-
-/**
- * Metadata description of a column including names, types and structure
- * information.
- */
-
-public interface ColumnMetadata {
-  enum StructureType {
-    PRIMITIVE, LIST, TUPLE
-  }
-
-  public static final int DEFAULT_ARRAY_SIZE = 10;
-
-  ColumnMetadata.StructureType structureType();
-  TupleMetadata mapSchema();
-  MaterializedField schema();
-  String name();
-  MajorType majorType();
-  MinorType type();
-  DataMode mode();
-  boolean isNullable();
-  boolean isArray();
-  boolean isVariableWidth();
-  boolean isMap();
-  boolean isList();
-
-  /**
-   * Report whether one column is equivalent to another. Columns are equivalent
-   * if they have the same name, type and structure (ignoring internal structure
-   * such as offset vectors.)
-   */
-
-  boolean isEquivalent(ColumnMetadata other);
-
-  /**
-   * For variable-width columns, specify the expected column width to be used
-   * when allocating a new vector. Does nothing for fixed-width columns.
-   *
-   * @param width the expected column width
-   */
-
-  void setExpectedWidth(int width);
-
-  /**
-   * Get the expected width for a column. This is the actual width for fixed-
-   * width columns, the specified width (defaulting to 50) for variable-width
-   * columns.
-   * @return the expected column width of the each data value. Does not include
-   * "overhead" space such as for the null-value vector or offset vector
-   */
-
-  int expectedWidth();
-
-  /**
-   * For an array column, specify the expected average array cardinality.
-   * Ignored for non-array columns. Used when allocating new vectors.
-   *
-   * @param childCount the expected average array cardinality. Defaults to
-   * 1 for non-array columns, 10 for array columns
-   */
-
-  void setExpectedElementCount(int childCount);
-
-  /**
-   * Returns the expected array cardinality for array columns, or 1 for
-   * non-array columns.
-   *
-   * @return the expected value cardinality per value (per-row for top-level
-   * columns, per array element for arrays within lists)
-   */
-
-  int expectedElementCount();
-
-  /**
-   * Create an empty version of this column. If the column is a scalar,
-   * produces a simple copy. If a map, produces a clone without child
-   * columns.
-   *
-   * @return empty clone of this column
-   */
-
-  ColumnMetadata cloneEmpty();
-
-  /**
-   * Reports whether, in this context, the column is projected outside
-   * of the context. (That is, whether the column is backed by an actual
-   * value vector.)
-   */
-
-  boolean isProjected();
-  void setProjected(boolean projected);
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/vector/src/main/java/org/apache/drill/exec/record/TupleMetadata.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/record/TupleMetadata.java b/exec/vector/src/main/java/org/apache/drill/exec/record/TupleMetadata.java
deleted file mode 100644
index 8f597be..0000000
--- a/exec/vector/src/main/java/org/apache/drill/exec/record/TupleMetadata.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.record;
-
-import java.util.List;
-
-/**
- * Metadata description of the schema of a row or a map.
- * In Drill, both rows and maps are
- * tuples: both are an ordered collection of values, defined by a
- * schema. Each tuple has a schema that defines the column ordering
- * for indexed access. Each tuple also provides methods to get column
- * accessors by name or index.
- * <p>
- * Models the physical schema of a row set showing the logical hierarchy of fields
- * with map fields as first-class fields. Map members appear as children
- * under the map, much as they appear in the physical value-vector
- * implementation.
- * <ul>
- * <li>Provides fast lookup by name or index.</li>
- * <li>Provides a nested schema, in this same form, for maps.</li>
- * </ul>
- * This form is useful when performing semantic analysis and when
- * working with vectors.
- * <p>
- * In the future, this structure will also gather metadata useful
- * for vector processing such as expected widths and so on.
- */
-
-public interface TupleMetadata extends Iterable<ColumnMetadata> {
-
-  /**
-   * Add a new column to the schema.
-   *
-   * @param columnSchema
-   * @return the index of the new column
-   */
-  ColumnMetadata add(MaterializedField field);
-  int addColumn(ColumnMetadata column);
-
-  int size();
-  boolean isEmpty();
-  int index(String name);
-  ColumnMetadata metadata(int index);
-  ColumnMetadata metadata(String name);
-  MaterializedField column(int index);
-  MaterializedField column(String name);
-  boolean isEquivalent(TupleMetadata other);
-  ColumnMetadata parent();
-
-  /**
-   * Return the schema as a list of <tt>MaterializedField</tt> objects
-   * which can be used to create other schemas. Not valid for a
-   * flattened schema.
-   *
-   * @return a list of the top-level fields. Maps contain their child
-   * fields
-   */
-
-  List<MaterializedField> toFieldList();
-
-  /**
-   * Full name of the column. Note: this name cannot be used to look up
-   * the column because of ambiguity. The name "a.b.c" may mean a single
-   * column with that name, or may mean maps "a", and "b" with column "c",
-   * etc.
-   *
-   * @return full, dotted, column name
-   */
-
-  String fullName(ColumnMetadata column);
-  String fullName(int index);
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/vector/src/main/java/org/apache/drill/exec/record/TupleNameSpace.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/record/TupleNameSpace.java b/exec/vector/src/main/java/org/apache/drill/exec/record/TupleNameSpace.java
deleted file mode 100644
index 5853c93..0000000
--- a/exec/vector/src/main/java/org/apache/drill/exec/record/TupleNameSpace.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.record;
-
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.drill.common.map.CaseInsensitiveMap;
-
-import com.google.common.collect.ImmutableList;
-
-/**
- * Implementation of a tuple name space. Tuples allow both indexed and
- * named access to their members.
- *
- * @param <T> the type of object representing each column
- */
-
-public class TupleNameSpace<T> implements Iterable<T> {
-  private final Map<String,Integer> nameSpace = CaseInsensitiveMap.newHashMap();
-  private final List<T> entries = new ArrayList<>();
-
-  public int add(String key, T value) {
-    if (indexOf(key) != -1) {
-      throw new IllegalArgumentException("Duplicate entry: " + key);
-    }
-    int index = entries.size();
-    nameSpace.put(key, index);
-    entries.add(value);
-    return index;
-  }
-
-  public T get(int index) {
-    return entries.get(index);
-  }
-
-  public T get(String key) {
-    int index = indexOf(key);
-    if (index == -1) {
-      return null;
-    }
-    return get(index);
-  }
-
-  public int indexOf(String key) {
-    Integer index = nameSpace.get(key);
-    if (index == null) {
-      return -1;
-    }
-    return index;
-  }
-
-  public int count() { return entries.size(); }
-
-  @Override
-  public Iterator<T> iterator() {
-    return entries.iterator();
-  }
-
-  public boolean isEmpty() {
-    return entries.isEmpty();
-  }
-
-  public List<T> entries() {
-    return ImmutableList.copyOf(entries);
-  }
-
-  @Override
-  public String toString() {
-    return entries.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/ColumnMetadata.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/ColumnMetadata.java b/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/ColumnMetadata.java
new file mode 100644
index 0000000..70a60e6
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/ColumnMetadata.java
@@ -0,0 +1,196 @@
+/*
+ * 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.record.metadata;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.record.MaterializedField;
+
+/**
+ * Metadata description of a column including names, types and structure
+ * information.
+ */
+
+public interface ColumnMetadata {
+
+  /**
+   * Rough characterization of Drill types into metadata categories.
+   * Various aspects of Drill's type system are very, very messy.
+   * However, Drill is defined by its code, not some abstract design,
+   * so the metadata system here does the best job it can to simplify
+   * the messy type system while staying close to the underlying
+   * implementation.
+   */
+
+  enum StructureType {
+
+    /**
+     * Primitive column (all types except List, Map and Union.)
+     * Includes (one-dimensional) arrays of those types.
+     */
+
+    PRIMITIVE,
+
+    /**
+     * Map or repeated map. Also describes the row as a whole.
+     */
+
+    TUPLE,
+
+    /**
+     * Union or (non-repeated) list. (A non-repeated list is,
+     * essentially, a repeated union.)
+     */
+
+    VARIANT,
+
+    /**
+     * A repeated list. A repeated list is not simply the repeated
+     * form of a list, it is something else entirely. It acts as
+     * a dimensional wrapper around any other type (except list)
+     * and adds a non-nullable extra dimension. Hence, this type is
+     * for 2D+ arrays.
+     * <p>
+     * In theory, a 2D list of, say, INT would be an INT column, but
+     * repeated in to dimensions. Alas, that is not how it is. Also,
+     * if we have a separate category for 2D lists, we should have
+     * a separate category for 1D lists. But, again, that is not how
+     * the code has evolved.
+     */
+
+    MULTI_ARRAY
+  }
+
+  int DEFAULT_ARRAY_SIZE = 10;
+
+  StructureType structureType();
+
+  /**
+   * Schema for <tt>TUPLE</tt> columns.
+   *
+   * @return the tuple schema
+   */
+
+  TupleMetadata mapSchema();
+
+  /**
+   * Schema for <tt>VARIANT</tt> columns.
+   *
+   * @return the variant schema
+   */
+
+  VariantMetadata variantSchema();
+
+  /**
+   * Schema of inner dimension for <tt>MULTI_ARRAY<tt> columns.
+   * If an array is 3D, the outer column represents all 3 dimensions.
+   * <tt>outer.childSchema()</tt> gives another <tt>MULTI_ARRAY</tt>
+   * for the inner 2D array.
+   * <tt>outer.childSchema().childSchema()</tt> gives a column
+   * of some other type (but repeated) for the 1D array.
+   * <p>
+   * Sorry for the mess, but it is how the code works and we are not
+   * in a position to revisit data type fundamentals.
+   *
+   * @return the description of the (n-1) st dimension.
+   */
+
+  ColumnMetadata childSchema();
+  MaterializedField schema();
+  MaterializedField emptySchema();
+  String name();
+  MinorType type();
+  MajorType majorType();
+  DataMode mode();
+  int dimensions();
+  boolean isNullable();
+  boolean isArray();
+  boolean isVariableWidth();
+  boolean isMap();
+  boolean isVariant();
+
+  /**
+   * Report whether one column is equivalent to another. Columns are equivalent
+   * if they have the same name, type and structure (ignoring internal structure
+   * such as offset vectors.)
+   */
+
+  boolean isEquivalent(ColumnMetadata other);
+
+  /**
+   * For variable-width columns, specify the expected column width to be used
+   * when allocating a new vector. Does nothing for fixed-width columns.
+   *
+   * @param width the expected column width
+   */
+
+  void setExpectedWidth(int width);
+
+  /**
+   * Get the expected width for a column. This is the actual width for fixed-
+   * width columns, the specified width (defaulting to 50) for variable-width
+   * columns.
+   * @return the expected column width of the each data value. Does not include
+   * "overhead" space such as for the null-value vector or offset vector
+   */
+
+  int expectedWidth();
+
+  /**
+   * For an array column, specify the expected average array cardinality.
+   * Ignored for non-array columns. Used when allocating new vectors.
+   *
+   * @param childCount the expected average array cardinality. Defaults to
+   * 1 for non-array columns, 10 for array columns
+   */
+
+  void setExpectedElementCount(int childCount);
+
+  /**
+   * Returns the expected array cardinality for array columns, or 1 for
+   * non-array columns.
+   *
+   * @return the expected value cardinality per value (per-row for top-level
+   * columns, per array element for arrays within lists)
+   */
+
+  int expectedElementCount();
+
+  /**
+   * Create an empty version of this column. If the column is a scalar,
+   * produces a simple copy. If a map, produces a clone without child
+   * columns.
+   *
+   * @return empty clone of this column
+   */
+
+  ColumnMetadata cloneEmpty();
+
+  /**
+   * Reports whether, in this context, the column is projected outside
+   * of the context. (That is, whether the column is backed by an actual
+   * value vector.)
+   */
+
+  boolean isProjected();
+  void setProjected(boolean projected);
+
+  int precision();
+  int scale();
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/ProjectionType.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/ProjectionType.java b/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/ProjectionType.java
new file mode 100644
index 0000000..4972605
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/ProjectionType.java
@@ -0,0 +1,27 @@
+/*
+ * 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.record.metadata;
+
+public enum ProjectionType {
+  UNPROJECTED,
+  WILDCARD,     // *
+  UNSPECIFIED,  // x
+  TUPLE,        // x.y
+  ARRAY,        // x[0]
+  TUPLE_ARRAY   // x[0].y
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/TupleMetadata.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/TupleMetadata.java b/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/TupleMetadata.java
new file mode 100644
index 0000000..a65c9f2
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/TupleMetadata.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.record.metadata;
+
+import java.util.List;
+
+import org.apache.drill.exec.record.MaterializedField;
+
+/**
+ * Metadata description of the schema of a row or a map.
+ * In Drill, both rows and maps are
+ * tuples: both are an ordered collection of values, defined by a
+ * schema. Each tuple has a schema that defines the column ordering
+ * for indexed access. Each tuple also provides methods to get column
+ * accessors by name or index.
+ * <p>
+ * Models the physical schema of a row set showing the logical hierarchy of fields
+ * with map fields as first-class fields. Map members appear as children
+ * under the map, much as they appear in the physical value-vector
+ * implementation.
+ * <ul>
+ * <li>Provides fast lookup by name or index.</li>
+ * <li>Provides a nested schema, in this same form, for maps.</li>
+ * </ul>
+ * This form is useful when performing semantic analysis and when
+ * working with vectors.
+ * <p>
+ * In the future, this structure will also gather metadata useful
+ * for vector processing such as expected widths and so on.
+ */
+
+public interface TupleMetadata extends Iterable<ColumnMetadata> {
+
+  /**
+   * Add a new column to the schema.
+   *
+   * @param columnSchema
+   * @return the index of the new column
+   */
+  ColumnMetadata add(MaterializedField field);
+  int addColumn(ColumnMetadata column);
+
+  int size();
+  boolean isEmpty();
+  int index(String name);
+  ColumnMetadata metadata(int index);
+  ColumnMetadata metadata(String name);
+  MaterializedField column(int index);
+  MaterializedField column(String name);
+  boolean isEquivalent(TupleMetadata other);
+  ColumnMetadata parent();
+
+  /**
+   * Return the schema as a list of <tt>MaterializedField</tt> objects
+   * which can be used to create other schemas. Not valid for a
+   * flattened schema.
+   *
+   * @return a list of the top-level fields. Maps contain their child
+   * fields
+   */
+
+  List<MaterializedField> toFieldList();
+
+  /**
+   * Full name of the column. Note: this name cannot be used to look up
+   * the column because of ambiguity. The name "a.b.c" may mean a single
+   * column with that name, or may mean maps "a", and "b" with column "c",
+   * etc.
+   *
+   * @return full, dotted, column name
+   */
+
+  String fullName(ColumnMetadata column);
+  String fullName(int index);
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/TupleNameSpace.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/TupleNameSpace.java b/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/TupleNameSpace.java
new file mode 100644
index 0000000..3379639
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/TupleNameSpace.java
@@ -0,0 +1,89 @@
+/*
+ * 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.record.metadata;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.drill.common.map.CaseInsensitiveMap;
+
+import com.google.common.collect.ImmutableList;
+
+/**
+ * Implementation of a tuple name space. Tuples allow both indexed and
+ * named access to their members.
+ *
+ * @param <T> the type of object representing each column
+ */
+
+public class TupleNameSpace<T> implements Iterable<T> {
+  private final Map<String,Integer> nameSpace = CaseInsensitiveMap.newHashMap();
+  private final List<T> entries = new ArrayList<>();
+
+  public int add(String key, T value) {
+    if (indexOf(key) != -1) {
+      throw new IllegalArgumentException("Duplicate entry: " + key);
+    }
+    int index = entries.size();
+    nameSpace.put(key, index);
+    entries.add(value);
+    return index;
+  }
+
+  public T get(int index) {
+    return entries.get(index);
+  }
+
+  public T get(String key) {
+    int index = indexOf(key);
+    if (index == -1) {
+      return null;
+    }
+    return get(index);
+  }
+
+  public int indexOf(String key) {
+    Integer index = nameSpace.get(key);
+    if (index == null) {
+      return -1;
+    }
+    return index;
+  }
+
+  public int count() { return entries.size(); }
+
+  @Override
+  public Iterator<T> iterator() {
+    return entries.iterator();
+  }
+
+  public boolean isEmpty() {
+    return entries.isEmpty();
+  }
+
+  public List<T> entries() {
+    return ImmutableList.copyOf(entries);
+  }
+
+  @Override
+  public String toString() {
+    return entries.toString();
+  }
+}


[05/17] drill git commit: DRILL-6174: Parquet filter pushdown improvements.

Posted by ar...@apache.org.
DRILL-6174: Parquet filter pushdown improvements.

Added support IS [NOT] NULL/TRUE/FALSE operator for the parquet filter pushdown.
Added timestamp/date/time implicit/explicit casts.

closes #1131


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/3bc4e319
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/3bc4e319
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/3bc4e319

Branch: refs/heads/master
Commit: 3bc4e31916fe3abddda8ab1c2504b0e67645a602
Parents: 1697e53
Author: Roman Kulyk <ro...@gmail.com>
Authored: Thu Feb 15 15:37:43 2018 +0000
Committer: Arina Ielchiieva <ar...@gmail.com>
Committed: Sat Mar 3 19:47:38 2018 +0200

----------------------------------------------------------------------
 .../exec/expr/fn/FunctionGenerationHelper.java  |   9 +-
 .../exec/expr/fn/impl/DateTypeFunctions.java    |  17 +-
 .../expr/stat/ParquetBooleanPredicates.java     |  75 ++++
 .../expr/stat/ParquetComparisonPredicates.java  | 292 +++++++++++++++
 .../exec/expr/stat/ParquetIsPredicates.java     | 209 +++++++++++
 .../drill/exec/expr/stat/ParquetPredicates.java | 352 -------------------
 .../exec/expr/stat/ParquetPredicatesHelper.java |  50 +++
 .../exec/expr/stat/RangeExprEvaluator.java      |  34 +-
 .../drill/exec/resolver/TypeCastRules.java      |   3 +-
 .../store/parquet/ParquetFilterBuilder.java     |  83 ++++-
 .../parquet/stat/ParquetMetaStatCollector.java  |   4 +
 .../parquet/TestParquetFilterPushDown.java      |  39 ++
 ...stParquetFilterPushDownForDateTimeCasts.java | 127 +++++++
 .../parquetFilterPush/blnTbl/0_0_0.parquet      | Bin 0 -> 272 bytes
 .../parquetFilterPush/blnTbl/0_0_1.parquet      | Bin 0 -> 272 bytes
 .../parquetFilterPush/blnTbl/0_0_2.parquet      | Bin 0 -> 272 bytes
 .../parquetFilterPush/blnTbl/0_0_3.parquet      | Bin 0 -> 259 bytes
 17 files changed, 925 insertions(+), 369 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/3bc4e319/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionGenerationHelper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionGenerationHelper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionGenerationHelper.java
index 435b451..7db862f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionGenerationHelper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionGenerationHelper.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -47,6 +47,13 @@ public class FunctionGenerationHelper {
   public static final String LT = "less_than";
   public static final String LE = "less_than_or_equal_to";
 
+  public static final String IS_NULL = "isnull";
+  public static final String IS_NOT_NULL = "isnotnull";
+  public static final String IS_TRUE = "istrue";
+  public static final String IS_NOT_TRUE = "isnottrue";
+  public static final String IS_FALSE = "isfalse";
+  public static final String IS_NOT_FALSE = "isnotfalse";
+
   /**
    * Finds ordering comparator ("compare_to...") FunctionHolderExpression with
    * a specified ordering for NULL (and considering NULLS <i>equal</i>).

http://git-wip-us.apache.org/repos/asf/drill/blob/3bc4e319/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DateTypeFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DateTypeFunctions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DateTypeFunctions.java
index a719ec0..25db420 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DateTypeFunctions.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DateTypeFunctions.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -524,6 +524,21 @@ public class DateTypeFunctions {
         }
     }
 
+    @FunctionTemplate(name = "castTIME", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+    public static class CastDateToTime implements DrillSimpleFunc {
+      @Param DateHolder in;
+      @Output TimeHolder out;
+
+      @Override
+      public void setup() {
+      }
+
+      @Override
+      public void eval() {
+        out.value = 0;
+      }
+    }
+
     @FunctionTemplate(name = "unix_timestamp", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
     public static class UnixTimeStamp implements DrillSimpleFunc {
       @Output BigIntHolder out;

http://git-wip-us.apache.org/repos/asf/drill/blob/3bc4e319/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetBooleanPredicates.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetBooleanPredicates.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetBooleanPredicates.java
new file mode 100644
index 0000000..9db629d
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetBooleanPredicates.java
@@ -0,0 +1,75 @@
+/*
+ * 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.expr.stat;
+
+import org.apache.drill.common.expression.BooleanOperator;
+import org.apache.drill.common.expression.ExpressionPosition;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.visitors.ExprVisitor;
+
+import java.util.List;
+
+/**
+ * Boolean predicates for parquet filter pushdown.
+ */
+public class ParquetBooleanPredicates {
+  public static abstract class ParquetBooleanPredicate extends BooleanOperator implements ParquetFilterPredicate {
+    public ParquetBooleanPredicate(String name, List<LogicalExpression> args, ExpressionPosition pos) {
+      super(name, args, pos);
+    }
+
+    @Override
+    public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E> visitor, V value) throws E {
+      return visitor.visitBooleanOperator(this, value);
+    }
+  }
+
+  public static class AndPredicate extends ParquetBooleanPredicate {
+    public AndPredicate(String name, List<LogicalExpression> args, ExpressionPosition pos) {
+      super(name, args, pos);
+    }
+
+    @Override
+    public boolean canDrop(RangeExprEvaluator evaluator) {
+      // "and" : as long as one branch is OK to drop, we can drop it.
+      for (LogicalExpression child : this) {
+        if (child instanceof ParquetFilterPredicate && ((ParquetFilterPredicate) child).canDrop(evaluator)) {
+          return true;
+        }
+      }
+      return false;
+    }
+  }
+
+  public static class OrPredicate extends ParquetBooleanPredicate {
+    public OrPredicate(String name, List<LogicalExpression> args, ExpressionPosition pos) {
+      super(name, args, pos);
+    }
+
+    @Override
+    public boolean canDrop(RangeExprEvaluator evaluator) {
+      for (LogicalExpression child : this) {
+        // "long" : as long as one branch is NOT ok to drop, we can NOT drop it.
+        if (! ((ParquetFilterPredicate) child).canDrop(evaluator)) {
+          return false;
+        }
+      }
+
+      return true;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/3bc4e319/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetComparisonPredicates.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetComparisonPredicates.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetComparisonPredicates.java
new file mode 100644
index 0000000..5657215
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetComparisonPredicates.java
@@ -0,0 +1,292 @@
+/*
+ * 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.expr.stat;
+
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.LogicalExpressionBase;
+import org.apache.drill.common.expression.visitors.ExprVisitor;
+import org.apache.parquet.column.statistics.Statistics;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * Comparison predicates for parquet filter pushdown.
+ */
+public class ParquetComparisonPredicates {
+  public static abstract  class ParquetCompPredicate extends LogicalExpressionBase implements ParquetFilterPredicate {
+    protected final LogicalExpression left;
+    protected final LogicalExpression right;
+
+    public ParquetCompPredicate(LogicalExpression left, LogicalExpression right) {
+      super(left.getPosition());
+      this.left = left;
+      this.right = right;
+    }
+
+    @Override
+    public Iterator<LogicalExpression> iterator() {
+      final List<LogicalExpression> args = new ArrayList<>();
+      args.add(left);
+      args.add(right);
+      return args.iterator();
+    }
+
+    @Override
+    public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E> visitor, V value) throws E {
+      return visitor.visitUnknown(this, value);
+    }
+
+  }
+
+  /**
+   * EQ (=) predicate
+   */
+  public static class EqualPredicate extends ParquetCompPredicate {
+    public EqualPredicate(LogicalExpression left, LogicalExpression right) {
+      super(left, right);
+    }
+
+    /**
+        Semantics of canDrop() is very similar to what is implemented in Parquet library's
+        {@link org.apache.parquet.filter2.statisticslevel.StatisticsFilter} and
+        {@link org.apache.parquet.filter2.predicate.FilterPredicate}
+
+        Main difference :
+     1. A RangeExprEvaluator is used to compute the min/max of an expression, such as CAST function
+        of a column. CAST function could be explicitly added by Drill user (It's recommended to use CAST
+        function after DRILL-4372, if user wants to reduce planning time for limit 0 query), or implicitly
+        inserted by Drill, when the types of compare operands are not identical. Therefore, it's important
+         to allow CAST function to appear in the filter predicate.
+     2. We do not require list of ColumnChunkMetaData to do the evaluation, while Parquet library's
+        StatisticsFilter has such requirement. Drill's ParquetTableMetaData does not maintain ColumnChunkMetaData,
+        making it impossible to directly use Parquet library's StatisticFilter in query planning time.
+     3. We allows both sides of comparison operator to be a min/max range. As such, we support
+           expression_of(Column1)   <   expression_of(Column2),
+        where Column1 and Column2 are from same parquet table.
+     */
+    @Override
+    public boolean canDrop(RangeExprEvaluator evaluator) {
+      Statistics leftStat = left.accept(evaluator, null);
+      Statistics rightStat = right.accept(evaluator, null);
+
+      if (leftStat == null ||
+          rightStat == null ||
+          leftStat.isEmpty() ||
+          rightStat.isEmpty()) {
+        return false;
+      }
+
+      // if either side is ALL null, = is evaluated to UNKNOW -> canDrop
+      if (ParquetPredicatesHelper.isAllNulls(leftStat, evaluator.getRowCount()) ||
+          ParquetPredicatesHelper.isAllNulls(rightStat, evaluator.getRowCount())) {
+        return true;
+      }
+
+      // can drop when left's max < right's min, or right's max < left's min
+      if ( ( leftStat.genericGetMax().compareTo(rightStat.genericGetMin()) < 0
+            || rightStat.genericGetMax().compareTo(leftStat.genericGetMin()) < 0)) {
+        return true;
+      } else {
+        return false;
+      }
+    }
+
+    @Override
+    public String toString() {
+      return left.toString()  + " = " + right.toString();
+    }
+  }
+
+  /**
+   * GT (>) predicate.
+   */
+  public static class GTPredicate extends ParquetCompPredicate {
+    public GTPredicate(LogicalExpression left, LogicalExpression right) {
+      super(left, right);
+    }
+
+    @Override
+    public boolean canDrop(RangeExprEvaluator evaluator) {
+      Statistics leftStat = left.accept(evaluator, null);
+      Statistics rightStat = right.accept(evaluator, null);
+
+      if (leftStat == null ||
+          rightStat == null ||
+          leftStat.isEmpty() ||
+          rightStat.isEmpty()) {
+        return false;
+      }
+
+      // if either side is ALL null, = is evaluated to UNKNOW -> canDrop
+      if (ParquetPredicatesHelper.isAllNulls(leftStat, evaluator.getRowCount()) ||
+          ParquetPredicatesHelper.isAllNulls(rightStat, evaluator.getRowCount())) {
+        return true;
+      }
+
+      // can drop when left's max <= right's min.
+      if ( leftStat.genericGetMax().compareTo(rightStat.genericGetMin()) <= 0 ) {
+        return true;
+      } else {
+        return false;
+      }
+    }
+  }
+
+  /**
+   * GE (>=) predicate.
+   */
+  public static class GEPredicate extends ParquetCompPredicate {
+    public GEPredicate(LogicalExpression left, LogicalExpression right) {
+      super(left, right);
+    }
+
+    @Override
+    public boolean canDrop(RangeExprEvaluator evaluator) {
+      Statistics leftStat = left.accept(evaluator, null);
+      Statistics rightStat = right.accept(evaluator, null);
+
+      if (leftStat == null ||
+          rightStat == null ||
+          leftStat.isEmpty() ||
+          rightStat.isEmpty()) {
+        return false;
+      }
+
+      // if either side is ALL null, = is evaluated to UNKNOW -> canDrop
+      if (ParquetPredicatesHelper.isAllNulls(leftStat, evaluator.getRowCount()) ||
+          ParquetPredicatesHelper.isAllNulls(rightStat, evaluator.getRowCount())) {
+        return true;
+      }
+
+      // can drop when left's max < right's min.
+      if ( leftStat.genericGetMax().compareTo(rightStat.genericGetMin()) < 0 ) {
+        return true;
+      } else {
+        return false;
+      }
+    }
+  }
+
+  /**
+   * LT (<) predicate.
+   */
+  public static class LTPredicate extends ParquetCompPredicate {
+    public LTPredicate(LogicalExpression left, LogicalExpression right) {
+      super(left, right);
+    }
+
+    @Override
+    public boolean canDrop(RangeExprEvaluator evaluator) {
+      Statistics leftStat = left.accept(evaluator, null);
+      Statistics rightStat = right.accept(evaluator, null);
+
+      if (leftStat == null ||
+          rightStat == null ||
+          leftStat.isEmpty() ||
+          rightStat.isEmpty()) {
+        return false;
+      }
+
+      // if either side is ALL null, = is evaluated to UNKNOW -> canDrop
+      if (ParquetPredicatesHelper.isAllNulls(leftStat, evaluator.getRowCount()) ||
+          ParquetPredicatesHelper.isAllNulls(rightStat, evaluator.getRowCount())) {
+        return true;
+      }
+
+      // can drop when right's max <= left's min.
+      if ( rightStat.genericGetMax().compareTo(leftStat.genericGetMin()) <= 0 ) {
+        return true;
+      } else {
+        return false;
+      }
+    }
+  }
+
+  /**
+   * LE (<=) predicate.
+   */
+  public static class LEPredicate extends ParquetCompPredicate {
+    public LEPredicate(LogicalExpression left, LogicalExpression right) {
+      super(left, right);
+    }
+
+    @Override
+    public boolean canDrop(RangeExprEvaluator evaluator) {
+      Statistics leftStat = left.accept(evaluator, null);
+      Statistics rightStat = right.accept(evaluator, null);
+
+      if (leftStat == null ||
+          rightStat == null ||
+          leftStat.isEmpty() ||
+          rightStat.isEmpty()) {
+        return false;
+      }
+
+      // if either side is ALL null, = is evaluated to UNKNOW -> canDrop
+      if (ParquetPredicatesHelper.isAllNulls(leftStat, evaluator.getRowCount()) ||
+          ParquetPredicatesHelper.isAllNulls(rightStat, evaluator.getRowCount())) {
+        return true;
+      }
+
+      // can drop when right's max < left's min.
+      if ( rightStat.genericGetMax().compareTo(leftStat.genericGetMin()) < 0 ) {
+        return true;
+      } else {
+        return false;
+      }
+    }
+  }
+
+  /**
+   * NE (!=) predicate.
+   */
+  public static class NEPredicate extends ParquetCompPredicate {
+    public NEPredicate(LogicalExpression left, LogicalExpression right) {
+      super(left, right);
+    }
+
+    @Override
+    public boolean canDrop(RangeExprEvaluator evaluator) {
+      Statistics leftStat = left.accept(evaluator, null);
+      Statistics rightStat = right.accept(evaluator, null);
+
+      if (leftStat == null ||
+          rightStat == null ||
+          leftStat.isEmpty() ||
+          rightStat.isEmpty()) {
+        return false;
+      }
+
+      // if either side is ALL null, comparison is evaluated to UNKNOW -> canDrop
+      if (ParquetPredicatesHelper.isAllNulls(leftStat, evaluator.getRowCount()) ||
+          ParquetPredicatesHelper.isAllNulls(rightStat, evaluator.getRowCount())) {
+        return true;
+      }
+
+      // can drop when there is only one unique value.
+      if ( leftStat.genericGetMin().compareTo(leftStat.genericGetMax()) == 0 &&
+           rightStat.genericGetMin().compareTo(rightStat.genericGetMax()) ==0 &&
+           leftStat.genericGetMax().compareTo(rightStat.genericGetMax()) == 0) {
+        return true;
+      } else {
+        return false;
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/3bc4e319/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetIsPredicates.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetIsPredicates.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetIsPredicates.java
new file mode 100644
index 0000000..c6f9b2f
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetIsPredicates.java
@@ -0,0 +1,209 @@
+/*
+ * 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.expr.stat;
+
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.LogicalExpressionBase;
+import org.apache.drill.common.expression.visitors.ExprVisitor;
+import org.apache.parquet.column.statistics.Statistics;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * IS predicates for parquet filter pushdown.
+ */
+public class ParquetIsPredicates {
+  public static abstract class ParquetIsPredicate extends LogicalExpressionBase implements ParquetFilterPredicate {
+    protected final LogicalExpression expr;
+
+    public ParquetIsPredicate(LogicalExpression expr) {
+      super(expr.getPosition());
+      this.expr = expr;
+    }
+
+    @Override
+    public Iterator<LogicalExpression> iterator() {
+      final List<LogicalExpression> args = new ArrayList<>();
+      args.add(expr);
+      return args.iterator();
+    }
+
+    @Override
+    public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E> visitor, V value) throws E {
+      return visitor.visitUnknown(this, value);
+    }
+  }
+
+  /**
+   * IS NULL predicate.
+   */
+  public static class IsNullPredicate extends ParquetIsPredicate {
+    public IsNullPredicate(LogicalExpression expr) {
+      super(expr);
+    }
+
+    @Override
+    public boolean canDrop(RangeExprEvaluator evaluator) {
+      Statistics exprStat = expr.accept(evaluator, null);
+
+      if (exprStat == null) {
+        return false;
+      }
+
+      //if there are no nulls  -> canDrop
+      if (!ParquetPredicatesHelper.hasNulls(exprStat)) {
+        return true;
+      } else {
+        return false;
+      }
+    }
+  }
+
+  /**
+   * IS NOT NULL predicate.
+   */
+  public static class IsNotNullPredicate extends ParquetIsPredicate {
+    public IsNotNullPredicate(LogicalExpression expr) {
+      super(expr);
+    }
+
+    @Override
+    public boolean canDrop(RangeExprEvaluator evaluator) {
+      Statistics exprStat = expr.accept(evaluator, null);
+
+      if (exprStat == null ||
+          exprStat.isEmpty()) {
+        return false;
+      }
+
+      //if there are all nulls  -> canDrop
+      if (ParquetPredicatesHelper.isAllNulls(exprStat, evaluator.getRowCount())) {
+        return true;
+      } else {
+        return false;
+      }
+    }
+  }
+
+  /**
+   * IS TRUE predicate.
+   */
+  public static class IsTruePredicate extends ParquetIsPredicate {
+    public IsTruePredicate(LogicalExpression expr) {
+      super(expr);
+    }
+
+    @Override
+    public boolean canDrop(RangeExprEvaluator evaluator) {
+      Statistics exprStat = expr.accept(evaluator, null);
+
+      if (exprStat == null ||
+          exprStat.isEmpty()) {
+        return false;
+      }
+
+      //if max value is not true or if there are all nulls  -> canDrop
+      if (exprStat.genericGetMax().compareTo(true) != 0 ||
+          ParquetPredicatesHelper.isAllNulls(exprStat, evaluator.getRowCount())) {
+        return true;
+      } else {
+        return false;
+      }
+    }
+  }
+
+  /**
+   * IS FALSE predicate.
+   */
+  public static class IsFalsePredicate extends ParquetIsPredicate {
+    public IsFalsePredicate(LogicalExpression expr) {
+      super(expr);
+    }
+
+    @Override
+    public boolean canDrop(RangeExprEvaluator evaluator) {
+      Statistics exprStat = expr.accept(evaluator, null);
+
+      if (exprStat == null ||
+          exprStat.isEmpty()) {
+        return false;
+      }
+
+      //if min value is not false or if there are all nulls  -> canDrop
+      if (exprStat.genericGetMin().compareTo(false) != 0 ||
+          ParquetPredicatesHelper.isAllNulls(exprStat, evaluator.getRowCount())) {
+        return true;
+      } else {
+        return false;
+      }
+    }
+  }
+
+  /**
+   * IS NOT TRUE predicate.
+   */
+  public static class IsNotTruePredicate extends ParquetIsPredicate {
+    public IsNotTruePredicate(LogicalExpression expr) {
+      super(expr);
+    }
+
+    @Override
+    public boolean canDrop(RangeExprEvaluator evaluator) {
+      Statistics exprStat = expr.accept(evaluator, null);
+
+      if (exprStat == null ||
+          exprStat.isEmpty()) {
+        return false;
+      }
+
+      //if min value is not false or if there are no nulls  -> canDrop
+      if (exprStat.genericGetMin().compareTo(false) != 0 && !ParquetPredicatesHelper.hasNulls(exprStat)) {
+        return true;
+      } else {
+        return false;
+      }
+    }
+  }
+
+  /**
+   * IS NOT FALSE predicate.
+   */
+  public static class IsNotFalsePredicate extends ParquetIsPredicate {
+    public IsNotFalsePredicate(LogicalExpression expr) {
+      super(expr);
+    }
+
+    @Override
+    public boolean canDrop(RangeExprEvaluator evaluator) {
+      Statistics exprStat = expr.accept(evaluator, null);
+
+      if (exprStat == null ||
+          exprStat.isEmpty()) {
+        return false;
+      }
+
+      //if max value is not true or if there are no nulls  -> canDrop
+      if (exprStat.genericGetMax().compareTo(true) != 0 && !ParquetPredicatesHelper.hasNulls(exprStat)) {
+        return true;
+      } else {
+        return false;
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/3bc4e319/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetPredicates.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetPredicates.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetPredicates.java
deleted file mode 100644
index 54f703a..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetPredicates.java
+++ /dev/null
@@ -1,352 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * 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.expr.stat;
-
-import org.apache.drill.common.expression.BooleanOperator;
-import org.apache.drill.common.expression.ExpressionPosition;
-import org.apache.drill.common.expression.LogicalExpression;
-import org.apache.drill.common.expression.LogicalExpressionBase;
-import org.apache.drill.common.expression.visitors.ExprVisitor;
-import org.apache.parquet.column.statistics.Statistics;
-import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
-
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
-public abstract  class ParquetPredicates {
-  public static abstract  class ParquetCompPredicate extends LogicalExpressionBase implements ParquetFilterPredicate {
-    protected final LogicalExpression left;
-    protected final LogicalExpression right;
-
-    public ParquetCompPredicate(LogicalExpression left, LogicalExpression right) {
-      super(left.getPosition());
-      this.left = left;
-      this.right = right;
-    }
-
-    @Override
-    public Iterator<LogicalExpression> iterator() {
-      final List<LogicalExpression> args = new ArrayList<>();
-      args.add(left);
-      args.add(right);
-      return args.iterator();
-    }
-
-    @Override
-    public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E> visitor, V value) throws E {
-      return visitor.visitUnknown(this, value);
-    }
-
-  }
-
-  public static abstract class ParquetBooleanPredicate extends BooleanOperator implements ParquetFilterPredicate {
-    public ParquetBooleanPredicate(String name, List<LogicalExpression> args, ExpressionPosition pos) {
-      super(name, args, pos);
-    }
-
-    @Override
-    public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E> visitor, V value) throws E {
-      return visitor.visitBooleanOperator(this, value);
-    }
-  }
-
-  public static class AndPredicate extends ParquetBooleanPredicate {
-    public AndPredicate(String name, List<LogicalExpression> args, ExpressionPosition pos) {
-      super(name, args, pos);
-    }
-
-    @Override
-    public boolean canDrop(RangeExprEvaluator evaluator) {
-      // "and" : as long as one branch is OK to drop, we can drop it.
-      for (LogicalExpression child : this) {
-        if (((ParquetFilterPredicate) child).canDrop(evaluator)) {
-          return true;
-        }
-      }
-      return false;
-    }
-  }
-
-  public static class OrPredicate extends ParquetBooleanPredicate {
-    public OrPredicate(String name, List<LogicalExpression> args, ExpressionPosition pos) {
-      super(name, args, pos);
-    }
-
-    @Override
-    public boolean canDrop(RangeExprEvaluator evaluator) {
-      for (LogicalExpression child : this) {
-        // "long" : as long as one branch is NOT ok to drop, we can NOT drop it.
-        if (! ((ParquetFilterPredicate) child).canDrop(evaluator)) {
-          return false;
-        }
-      }
-
-      return true;
-    }
-  }
-
-  // is this column chunk composed entirely of nulls?
-  // assumes the column chunk's statistics is not empty
-  protected static boolean isAllNulls(Statistics stat, long rowCount) {
-    return stat.getNumNulls() == rowCount;
-  }
-
-  // are there any nulls in this column chunk?
-  // assumes the column chunk's statistics is not empty
-  protected static boolean hasNulls(Statistics stat) {
-    return stat.getNumNulls() > 0;
-  }
-
-  /**
-   * EQ (=) predicate
-   */
-  public static class EqualPredicate extends ParquetCompPredicate {
-    public EqualPredicate(LogicalExpression left, LogicalExpression right) {
-      super(left, right);
-    }
-
-    /**
-        Semantics of canDrop() is very similar to what is implemented in Parquet library's
-        {@link org.apache.parquet.filter2.statisticslevel.StatisticsFilter} and
-        {@link org.apache.parquet.filter2.predicate.FilterPredicate}
-
-        Main difference :
-     1. A RangeExprEvaluator is used to compute the min/max of an expression, such as CAST function
-        of a column. CAST function could be explicitly added by Drill user (It's recommended to use CAST
-        function after DRILL-4372, if user wants to reduce planning time for limit 0 query), or implicitly
-        inserted by Drill, when the types of compare operands are not identical. Therefore, it's important
-         to allow CAST function to appear in the filter predicate.
-     2. We do not require list of ColumnChunkMetaData to do the evaluation, while Parquet library's
-        StatisticsFilter has such requirement. Drill's ParquetTableMetaData does not maintain ColumnChunkMetaData,
-        making it impossible to directly use Parquet library's StatisticFilter in query planning time.
-     3. We allows both sides of comparison operator to be a min/max range. As such, we support
-           expression_of(Column1)   <   expression_of(Column2),
-        where Column1 and Column2 are from same parquet table.
-     */
-    @Override
-    public boolean canDrop(RangeExprEvaluator evaluator) {
-      Statistics leftStat = left.accept(evaluator, null);
-      Statistics rightStat = right.accept(evaluator, null);
-
-      if (leftStat == null ||
-          rightStat == null ||
-          leftStat.isEmpty() ||
-          rightStat.isEmpty()) {
-        return false;
-      }
-
-      // if either side is ALL null, = is evaluated to UNKNOW -> canDrop
-      if (isAllNulls(leftStat, evaluator.getRowCount()) ||
-          isAllNulls(rightStat, evaluator.getRowCount())) {
-        return true;
-      }
-
-      // can drop when left's max < right's min, or right's max < left's min
-      if ( ( leftStat.genericGetMax().compareTo(rightStat.genericGetMin()) < 0
-            || rightStat.genericGetMax().compareTo(leftStat.genericGetMin()) < 0)) {
-        return true;
-      } else {
-        return false;
-      }
-    }
-
-    @Override
-    public String toString() {
-      return left.toString()  + " = " + right.toString();
-    }
-  }
-
-  /**
-   * GT (>) predicate.
-   */
-  public static class GTPredicate extends ParquetCompPredicate {
-    public GTPredicate(LogicalExpression left, LogicalExpression right) {
-      super(left, right);
-    }
-
-    @Override
-    public boolean canDrop(RangeExprEvaluator evaluator) {
-      Statistics leftStat = left.accept(evaluator, null);
-      Statistics rightStat = right.accept(evaluator, null);
-
-      if (leftStat == null ||
-          rightStat == null ||
-          leftStat.isEmpty() ||
-          rightStat.isEmpty()) {
-        return false;
-      }
-
-      // if either side is ALL null, = is evaluated to UNKNOW -> canDrop
-      if (isAllNulls(leftStat, evaluator.getRowCount()) ||
-          isAllNulls(rightStat, evaluator.getRowCount())) {
-        return true;
-      }
-
-      // can drop when left's max <= right's min.
-      if ( leftStat.genericGetMax().compareTo(rightStat.genericGetMin()) <= 0 ) {
-        return true;
-      } else {
-        return false;
-      }
-    }
-  }
-
-  /**
-   * GE (>=) predicate.
-   */
-  public static class GEPredicate extends ParquetCompPredicate {
-    public GEPredicate(LogicalExpression left, LogicalExpression right) {
-      super(left, right);
-    }
-
-    @Override
-    public boolean canDrop(RangeExprEvaluator evaluator) {
-      Statistics leftStat = left.accept(evaluator, null);
-      Statistics rightStat = right.accept(evaluator, null);
-
-      if (leftStat == null ||
-          rightStat == null ||
-          leftStat.isEmpty() ||
-          rightStat.isEmpty()) {
-        return false;
-      }
-
-      // if either side is ALL null, = is evaluated to UNKNOW -> canDrop
-      if (isAllNulls(leftStat, evaluator.getRowCount()) ||
-          isAllNulls(rightStat, evaluator.getRowCount())) {
-        return true;
-      }
-
-      // can drop when left's max < right's min.
-      if ( leftStat.genericGetMax().compareTo(rightStat.genericGetMin()) < 0 ) {
-        return true;
-      } else {
-        return false;
-      }
-    }
-  }
-
-  /**
-   * LT (<) predicate.
-   */
-  public static class LTPredicate extends ParquetCompPredicate {
-    public LTPredicate(LogicalExpression left, LogicalExpression right) {
-      super(left, right);
-    }
-
-    @Override
-    public boolean canDrop(RangeExprEvaluator evaluator) {
-      Statistics leftStat = left.accept(evaluator, null);
-      Statistics rightStat = right.accept(evaluator, null);
-
-      if (leftStat == null ||
-          rightStat == null ||
-          leftStat.isEmpty() ||
-          rightStat.isEmpty()) {
-        return false;
-      }
-
-      // if either side is ALL null, = is evaluated to UNKNOW -> canDrop
-      if (isAllNulls(leftStat, evaluator.getRowCount()) ||
-          isAllNulls(rightStat, evaluator.getRowCount())) {
-        return true;
-      }
-
-      // can drop when right's max <= left's min.
-      if ( rightStat.genericGetMax().compareTo(leftStat.genericGetMin()) <= 0 ) {
-        return true;
-      } else {
-        return false;
-      }
-    }
-  }
-
-  /**
-   * LE (<=) predicate.
-   */
-  public static class LEPredicate extends ParquetCompPredicate {
-    public LEPredicate(LogicalExpression left, LogicalExpression right) {
-      super(left, right);
-    }
-
-    @Override
-    public boolean canDrop(RangeExprEvaluator evaluator) {
-      Statistics leftStat = left.accept(evaluator, null);
-      Statistics rightStat = right.accept(evaluator, null);
-
-      if (leftStat == null ||
-          rightStat == null ||
-          leftStat.isEmpty() ||
-          rightStat.isEmpty()) {
-        return false;
-      }
-
-      // if either side is ALL null, = is evaluated to UNKNOW -> canDrop
-      if (isAllNulls(leftStat, evaluator.getRowCount()) ||
-          isAllNulls(rightStat, evaluator.getRowCount())) {
-        return true;
-      }
-
-      // can drop when right's max < left's min.
-      if ( rightStat.genericGetMax().compareTo(leftStat.genericGetMin()) < 0 ) {
-        return true;
-      } else {
-        return false;
-      }
-    }
-  }
-
-  /**
-   * NE (!=) predicate.
-   */
-  public static class NEPredicate extends ParquetCompPredicate {
-    public NEPredicate(LogicalExpression left, LogicalExpression right) {
-      super(left, right);
-    }
-
-    @Override
-    public boolean canDrop(RangeExprEvaluator evaluator) {
-      Statistics leftStat = left.accept(evaluator, null);
-      Statistics rightStat = right.accept(evaluator, null);
-
-      if (leftStat == null ||
-          rightStat == null ||
-          leftStat.isEmpty() ||
-          rightStat.isEmpty()) {
-        return false;
-      }
-
-      // if either side is ALL null, comparison is evaluated to UNKNOW -> canDrop
-      if (isAllNulls(leftStat, evaluator.getRowCount()) ||
-          isAllNulls(rightStat, evaluator.getRowCount())) {
-        return true;
-      }
-
-      // can drop when there is only one unique value.
-      if ( leftStat.genericGetMin().compareTo(leftStat.genericGetMax()) == 0 &&
-           rightStat.genericGetMin().compareTo(rightStat.genericGetMax()) ==0 &&
-           leftStat.genericGetMax().compareTo(rightStat.genericGetMax()) == 0) {
-        return true;
-      } else {
-        return false;
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/3bc4e319/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetPredicatesHelper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetPredicatesHelper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetPredicatesHelper.java
new file mode 100644
index 0000000..ac82d65
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetPredicatesHelper.java
@@ -0,0 +1,50 @@
+/*
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.expr.stat;
+
+import org.apache.parquet.column.statistics.Statistics;
+
+/**
+ * Parquet predicates class helper for filter pushdown.
+ */
+public class ParquetPredicatesHelper {
+
+  /**
+   * Checks that column chunk's statistics has only nulls
+   *
+   * @param stat parquet column statistics
+   * @param rowCount number of rows in the parquet file
+   * @return True if all rows are null in the parquet file
+   *          False if at least one row is not null.
+   */
+  public static boolean isAllNulls(Statistics stat, long rowCount) {
+    return stat.getNumNulls() == rowCount;
+  }
+
+  /**
+   * Checks that column chunk's statistics has at least one null
+   *
+   * @param stat parquet column statistics
+   * @return True if the parquet file has nulls
+   *          False if the parquet file hasn't nulls.
+   */
+  public static boolean hasNulls(Statistics stat) {
+    return stat.getNumNulls() > 0;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/3bc4e319/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/RangeExprEvaluator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/RangeExprEvaluator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/RangeExprEvaluator.java
index 8f77070..2d241dc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/RangeExprEvaluator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/RangeExprEvaluator.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -35,9 +35,11 @@ import org.apache.drill.exec.expr.holders.BigIntHolder;
 import org.apache.drill.exec.expr.holders.Float4Holder;
 import org.apache.drill.exec.expr.holders.Float8Holder;
 import org.apache.drill.exec.expr.holders.IntHolder;
+import org.apache.drill.exec.expr.holders.TimeStampHolder;
 import org.apache.drill.exec.expr.holders.ValueHolder;
 import org.apache.drill.exec.store.parquet.stat.ColumnStatistics;
 import org.apache.drill.exec.vector.ValueHolderHelper;
+import org.apache.parquet.column.statistics.BooleanStatistics;
 import org.apache.parquet.column.statistics.DoubleStatistics;
 import org.apache.parquet.column.statistics.FloatStatistics;
 import org.apache.parquet.column.statistics.IntStatistics;
@@ -73,9 +75,8 @@ public class RangeExprEvaluator extends AbstractExprVisitor<Statistics, Void, Ru
       final ColumnStatistics columnStatistics = columnStatMap.get(fieldExpr.getPath());
       if (columnStatistics != null) {
         return columnStatistics.getStatistics();
-      } else {
+      } else if (fieldExpr.getMajorType().equals(Types.OPTIONAL_INT)) {
         // field does not exist.
-        Preconditions.checkArgument(fieldExpr.getMajorType().equals(Types.OPTIONAL_INT));
         IntStatistics intStatistics = new IntStatistics();
         intStatistics.setNumNulls(rowCount); // all values are nulls
         return intStatistics;
@@ -90,6 +91,11 @@ public class RangeExprEvaluator extends AbstractExprVisitor<Statistics, Void, Ru
   }
 
   @Override
+  public Statistics visitBooleanConstant(ValueExpressions.BooleanExpression expr, Void value) throws RuntimeException {
+    return getStatistics(expr.getBoolean());
+  }
+
+  @Override
   public Statistics visitLongConstant(ValueExpressions.LongExpression expr, Void value) throws RuntimeException {
     return getStatistics(expr.getLong());
   }
@@ -152,6 +158,16 @@ public class RangeExprEvaluator extends AbstractExprVisitor<Statistics, Void, Ru
     return intStatistics;
   }
 
+  private BooleanStatistics getStatistics(boolean value) {
+    return getStatistics(value, value);
+  }
+
+  private BooleanStatistics getStatistics(boolean min, boolean max) {
+    final BooleanStatistics booleanStatistics = new BooleanStatistics();
+    booleanStatistics.setMinMax(min, max);
+    return booleanStatistics;
+  }
+
   private LongStatistics getStatistics(long value) {
     return getStatistics(value, value);
   }
@@ -217,6 +233,10 @@ public class RangeExprEvaluator extends AbstractExprVisitor<Statistics, Void, Ru
         minHolder = ValueHolderHelper.getFloat8Holder(((DoubleStatistics)input).getMin());
         maxHolder = ValueHolderHelper.getFloat8Holder(((DoubleStatistics)input).getMax());
         break;
+      case DATE:
+        minHolder = ValueHolderHelper.getDateHolder(((LongStatistics)input).getMin());
+        maxHolder = ValueHolderHelper.getDateHolder(((LongStatistics)input).getMax());
+        break;
       default:
         return null;
       }
@@ -237,6 +257,8 @@ public class RangeExprEvaluator extends AbstractExprVisitor<Statistics, Void, Ru
         return getStatistics( ((Float4Holder)minFuncHolder).value, ((Float4Holder)maxFuncHolder).value);
       case FLOAT8:
         return getStatistics( ((Float8Holder)minFuncHolder).value, ((Float8Holder)maxFuncHolder).value);
+      case TIMESTAMP:
+        return getStatistics(((TimeStampHolder) minFuncHolder).value, ((TimeStampHolder) maxFuncHolder).value);
       default:
         return null;
       }
@@ -245,7 +267,7 @@ public class RangeExprEvaluator extends AbstractExprVisitor<Statistics, Void, Ru
     }
   }
 
-  static Map<TypeProtos.MinorType, Set<TypeProtos.MinorType>> CAST_FUNC = new HashMap<>();
+  private static final Map<TypeProtos.MinorType, Set<TypeProtos.MinorType>> CAST_FUNC = new HashMap<>();
   static {
     // float -> double , int, bigint
     CAST_FUNC.put(TypeProtos.MinorType.FLOAT4, new HashSet<TypeProtos.MinorType>());
@@ -270,6 +292,10 @@ public class RangeExprEvaluator extends AbstractExprVisitor<Statistics, Void, Ru
     CAST_FUNC.get(TypeProtos.MinorType.BIGINT).add(TypeProtos.MinorType.INT);
     CAST_FUNC.get(TypeProtos.MinorType.BIGINT).add(TypeProtos.MinorType.FLOAT4);
     CAST_FUNC.get(TypeProtos.MinorType.BIGINT).add(TypeProtos.MinorType.FLOAT8);
+
+    // date -> timestamp
+    CAST_FUNC.put(TypeProtos.MinorType.DATE, new HashSet<TypeProtos.MinorType>());
+    CAST_FUNC.get(TypeProtos.MinorType.DATE).add(TypeProtos.MinorType.TIMESTAMP);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/3bc4e319/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/TypeCastRules.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/TypeCastRules.java b/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/TypeCastRules.java
index 78a4509..f0e6602 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/TypeCastRules.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/TypeCastRules.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -405,6 +405,7 @@ public class TypeCastRules {
     /** TIME cast able from **/
     rule = new HashSet<>();
     rule.add(MinorType.TIME);
+    rule.add(MinorType.DATE);
     rule.add(MinorType.TIMESTAMP);
     rule.add(MinorType.TIMESTAMPTZ);
     rule.add(MinorType.FIXEDCHAR);

http://git-wip-us.apache.org/repos/asf/drill/blob/3bc4e319/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFilterBuilder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFilterBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFilterBuilder.java
index 37a57dc..a9e55dd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFilterBuilder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFilterBuilder.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -30,6 +30,7 @@ import org.apache.drill.exec.expr.fn.DrillSimpleFuncHolder;
 import org.apache.drill.exec.expr.fn.FunctionGenerationHelper;
 import org.apache.drill.exec.expr.fn.interpreter.InterpreterEvaluator;
 import org.apache.drill.exec.expr.holders.BigIntHolder;
+import org.apache.drill.exec.expr.holders.BitHolder;
 import org.apache.drill.exec.expr.holders.DateHolder;
 import org.apache.drill.exec.expr.holders.Float4Holder;
 import org.apache.drill.exec.expr.holders.Float8Holder;
@@ -37,7 +38,9 @@ import org.apache.drill.exec.expr.holders.IntHolder;
 import org.apache.drill.exec.expr.holders.TimeHolder;
 import org.apache.drill.exec.expr.holders.TimeStampHolder;
 import org.apache.drill.exec.expr.holders.ValueHolder;
-import org.apache.drill.exec.expr.stat.ParquetPredicates;
+import org.apache.drill.exec.expr.stat.ParquetBooleanPredicates;
+import org.apache.drill.exec.expr.stat.ParquetComparisonPredicates;
+import org.apache.drill.exec.expr.stat.ParquetIsPredicates;
 import org.apache.drill.exec.expr.stat.TypedFieldExpr;
 import org.apache.drill.exec.ops.UdfUtilities;
 import org.slf4j.Logger;
@@ -124,6 +127,11 @@ public class ParquetFilterBuilder extends AbstractExprVisitor<LogicalExpression,
   }
 
   @Override
+  public LogicalExpression visitBooleanConstant(ValueExpressions.BooleanExpression booleanExpression, Set<LogicalExpression> value) throws RuntimeException {
+    return booleanExpression;
+  }
+
+  @Override
   public LogicalExpression visitBooleanOperator(BooleanOperator op, Set<LogicalExpression> value) {
     List<LogicalExpression> childPredicates = new ArrayList<>();
     String functionName = op.getName();
@@ -146,9 +154,9 @@ public class ParquetFilterBuilder extends AbstractExprVisitor<LogicalExpression,
       return childPredicates.get(0); // only one leg is qualified, remove boolean op.
     } else {
       if (functionName.equals("booleanOr")) {
-        return new ParquetPredicates.OrPredicate(op.getName(), childPredicates, op.getPosition());
+        return new ParquetBooleanPredicates.OrPredicate(op.getName(), childPredicates, op.getPosition());
       } else {
-        return new ParquetPredicates.AndPredicate(op.getName(), childPredicates, op.getPosition());
+        return new ParquetBooleanPredicates.AndPredicate(op.getName(), childPredicates, op.getPosition());
       }
     }
   }
@@ -181,6 +189,8 @@ public class ParquetFilterBuilder extends AbstractExprVisitor<LogicalExpression,
       return ValueExpressions.getTimeStamp(((TimeStampHolder) holder).value);
     case TIME:
       return ValueExpressions.getTime(((TimeHolder) holder).value);
+    case BIT:
+      return ValueExpressions.getBit(((BitHolder) holder).value == 1);
     default:
       return null;
     }
@@ -214,6 +224,10 @@ public class ParquetFilterBuilder extends AbstractExprVisitor<LogicalExpression,
       return handleCompareFunction(funcHolderExpr, value);
     }
 
+    if (isIsFunction(funcName)) {
+      return handleIsFunction(funcHolderExpr, value);
+    }
+
     if (CastFunctions.isCastFunction(funcName)) {
       List<LogicalExpression> newArgs = new ArrayList();
       for (LogicalExpression arg : funcHolderExpr.args) {
@@ -245,22 +259,53 @@ public class ParquetFilterBuilder extends AbstractExprVisitor<LogicalExpression,
 
     switch (funcName) {
     case FunctionGenerationHelper.EQ :
-      return new ParquetPredicates.EqualPredicate(newArgs.get(0), newArgs.get(1));
+      return new ParquetComparisonPredicates.EqualPredicate(newArgs.get(0), newArgs.get(1));
     case FunctionGenerationHelper.GT :
-      return new ParquetPredicates.GTPredicate(newArgs.get(0), newArgs.get(1));
+      return new ParquetComparisonPredicates.GTPredicate(newArgs.get(0), newArgs.get(1));
     case FunctionGenerationHelper.GE :
-      return new ParquetPredicates.GEPredicate(newArgs.get(0), newArgs.get(1));
+      return new ParquetComparisonPredicates.GEPredicate(newArgs.get(0), newArgs.get(1));
     case FunctionGenerationHelper.LT :
-      return new ParquetPredicates.LTPredicate(newArgs.get(0), newArgs.get(1));
+      return new ParquetComparisonPredicates.LTPredicate(newArgs.get(0), newArgs.get(1));
     case FunctionGenerationHelper.LE :
-      return new ParquetPredicates.LEPredicate(newArgs.get(0), newArgs.get(1));
+      return new ParquetComparisonPredicates.LEPredicate(newArgs.get(0), newArgs.get(1));
     case FunctionGenerationHelper.NE :
-      return new ParquetPredicates.NEPredicate(newArgs.get(0), newArgs.get(1));
+      return new ParquetComparisonPredicates.NEPredicate(newArgs.get(0), newArgs.get(1));
     default:
       return null;
     }
   }
 
+  private LogicalExpression handleIsFunction(FunctionHolderExpression functionHolderExpression, Set<LogicalExpression> value) {
+    String funcName;
+
+    if (functionHolderExpression.getHolder() instanceof DrillSimpleFuncHolder) {
+      funcName = ((DrillSimpleFuncHolder) functionHolderExpression.getHolder()).getRegisteredNames()[0];
+    } else {
+      logger.warn("Can not cast {} to DrillSimpleFuncHolder. Parquet filter pushdown can not handle function.",
+          functionHolderExpression.getHolder());
+      return null;
+    }
+    LogicalExpression arg = functionHolderExpression.args.get(0);
+
+    switch (funcName) {
+      case FunctionGenerationHelper.IS_NULL:
+        return new ParquetIsPredicates.IsNullPredicate(arg.accept(this, value));
+      case FunctionGenerationHelper.IS_NOT_NULL:
+        return new ParquetIsPredicates.IsNotNullPredicate(arg.accept(this, value));
+      case FunctionGenerationHelper.IS_TRUE:
+        return new ParquetIsPredicates.IsTruePredicate(arg.accept(this, value));
+      case FunctionGenerationHelper.IS_NOT_TRUE:
+        return new ParquetIsPredicates.IsNotTruePredicate(arg.accept(this, value));
+      case FunctionGenerationHelper.IS_FALSE:
+        return new ParquetIsPredicates.IsFalsePredicate(arg.accept(this, value));
+      case FunctionGenerationHelper.IS_NOT_FALSE:
+        return new ParquetIsPredicates.IsNotFalsePredicate(arg.accept(this, value));
+      default:
+        logger.warn("Unhandled IS function. Function name: {}", funcName);
+        return null;
+    }
+  }
+
   private LogicalExpression handleCastFunction(FunctionHolderExpression functionHolderExpression, Set<LogicalExpression> value) {
     for (LogicalExpression arg : functionHolderExpression.args) {
       LogicalExpression newArg = arg.accept(this, value);
@@ -278,6 +323,10 @@ public class ParquetFilterBuilder extends AbstractExprVisitor<LogicalExpression,
     return COMPARE_FUNCTIONS_SET.contains(funcName);
   }
 
+  private static boolean isIsFunction(String funcName) {
+    return IS_FUNCTIONS_SET.contains(funcName);
+  }
+
   private static final ImmutableSet<String> COMPARE_FUNCTIONS_SET;
 
   static {
@@ -292,4 +341,18 @@ public class ParquetFilterBuilder extends AbstractExprVisitor<LogicalExpression,
         .build();
   }
 
+  private static final ImmutableSet<String> IS_FUNCTIONS_SET;
+
+  static {
+    ImmutableSet.Builder<String> builder = ImmutableSet.builder();
+    IS_FUNCTIONS_SET = builder
+        .add(FunctionGenerationHelper.IS_NULL)
+        .add(FunctionGenerationHelper.IS_NOT_NULL)
+        .add(FunctionGenerationHelper.IS_TRUE)
+        .add(FunctionGenerationHelper.IS_NOT_TRUE)
+        .add(FunctionGenerationHelper.IS_FALSE)
+        .add(FunctionGenerationHelper.IS_NOT_FALSE)
+        .build();
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/3bc4e319/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/stat/ParquetMetaStatCollector.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/stat/ParquetMetaStatCollector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/stat/ParquetMetaStatCollector.java
index 4501cb8..4991a22 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/stat/ParquetMetaStatCollector.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/stat/ParquetMetaStatCollector.java
@@ -24,6 +24,7 @@ import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.store.parquet.Metadata;
 import org.apache.drill.exec.store.parquet.ParquetGroupScan;
 import org.apache.parquet.column.statistics.BinaryStatistics;
+import org.apache.parquet.column.statistics.BooleanStatistics;
 import org.apache.parquet.column.statistics.DoubleStatistics;
 import org.apache.parquet.column.statistics.FloatStatistics;
 import org.apache.parquet.column.statistics.IntStatistics;
@@ -180,6 +181,9 @@ public class ParquetMetaStatCollector implements  ColumnStatCollector{
         final long maxMS = convertToDrillDateValue(Integer.parseInt(max.toString()));
         ((LongStatistics) convertedStat ).setMinMax(minMS, maxMS);
         break;
+      case BIT:
+        ((BooleanStatistics) stat).setMinMax(Boolean.parseBoolean(min.toString()), Boolean.parseBoolean(max.toString()));
+        break;
       default:
       }
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/3bc4e319/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDown.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDown.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDown.java
index 98e7ef2..606d409 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDown.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDown.java
@@ -388,6 +388,45 @@ public class TestParquetFilterPushDown extends PlanTestBase {
     testParquetFilterPD(query3, 49, 2, false);
   }
 
+  @Test
+  public void testBooleanPredicate() throws Exception {
+    // Table blnTbl was created by CTAS in drill 1.12.0 and consist of 4 files withe the next data:
+    //    File 0_0_0.parquet has col_bln column with the next values: true, true, true.
+    //    File 0_0_1.parquet has col_bln column with the next values: false, false, false.
+    //    File 0_0_2.parquet has col_bln column with the next values: true, null, false.
+    //    File 0_0_3.parquet has col_bln column with the next values: null, null, null.
+
+    final String queryIsNull = "select col_bln from dfs.`parquetFilterPush/blnTbl` where col_bln is null";
+    testParquetFilterPD(queryIsNull, 4, 2, false);
+
+    final String queryIsNotNull = "select col_bln from dfs.`parquetFilterPush/blnTbl` where col_bln is not null";
+    testParquetFilterPD(queryIsNotNull, 8, 3, false);
+
+    final String queryIsTrue = "select col_bln from dfs.`parquetFilterPush/blnTbl` where col_bln is true";
+    testParquetFilterPD(queryIsTrue, 4, 2, false);
+
+    final String queryIsNotTrue = "select col_bln from dfs.`parquetFilterPush/blnTbl` where col_bln is not true";
+    testParquetFilterPD(queryIsNotTrue, 8, 3, false);
+
+    final String queryIsFalse = "select col_bln from dfs.`parquetFilterPush/blnTbl` where col_bln is false";
+    testParquetFilterPD(queryIsFalse, 4, 2, false);
+
+    final String queryIsNotFalse = "select col_bln from dfs.`parquetFilterPush/blnTbl` where col_bln is not false";
+    testParquetFilterPD(queryIsNotFalse, 8, 3, false);
+
+    final String queryEqualTrue = "select col_bln from dfs.`parquetFilterPush/blnTbl` where col_bln = true";
+    testParquetFilterPD(queryEqualTrue, 4, 2, false);
+
+    final String queryNotEqualTrue = "select col_bln from dfs.`parquetFilterPush/blnTbl` where not col_bln = true";
+    testParquetFilterPD(queryNotEqualTrue, 4, 2, false);
+
+    final String queryEqualFalse = "select col_bln from dfs.`parquetFilterPush/blnTbl` where col_bln = false";
+    testParquetFilterPD(queryEqualFalse, 4, 2, false);
+
+    final String queryNotEqualFalse = "select col_bln from dfs.`parquetFilterPush/blnTbl` where not col_bln = false";
+    testParquetFilterPD(queryNotEqualFalse, 4, 2, false);
+  }
+
   @Test // DRILL-5359
   public void testFilterWithItemFlatten() throws Exception {
     final String sql = "select n_regionkey\n"

http://git-wip-us.apache.org/repos/asf/drill/blob/3bc4e319/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDownForDateTimeCasts.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDownForDateTimeCasts.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDownForDateTimeCasts.java
new file mode 100644
index 0000000..ae3bac0
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDownForDateTimeCasts.java
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.store.parquet;
+
+import org.apache.drill.PlanTestBase;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestParquetFilterPushDownForDateTimeCasts extends PlanTestBase {
+
+  private static final String TABLE_NAME = "dateTimeCasts";
+
+  @BeforeClass
+  public static void init() throws Exception {
+    test("use dfs.tmp");
+    test("create table `%s/p1` as\n" +
+        "select timestamp '2017-01-01 00:00:00' as col_timestamp, date '2017-01-01' as col_date, time '00:00:00' as col_time from (values(1)) union all\n" +
+        "select timestamp '2017-01-02 00:00:00' as col_timestamp, date '2017-01-02' as col_date, time '00:00:00' as col_time from (values(1)) union all\n" +
+        "select timestamp '2017-01-02 21:01:15' as col_timestamp, date '2017-01-02' as col_date, time '21:01:15' as col_time from (values(1))", TABLE_NAME);
+
+    test("create table `%s/p2` as\n" +
+        "select timestamp '2017-01-03 08:50:00' as col_timestamp, date '2017-01-03' as col_date, time '08:50:00' as col_time from (values(1)) union all\n" +
+        "select timestamp '2017-01-04 15:25:00' as col_timestamp, date '2017-01-04' as col_date, time '15:25:00' as col_time from (values(1)) union all\n" +
+        "select timestamp '2017-01-04 22:14:29' as col_timestamp, date '2017-01-04' as col_date, time '22:14:29' as col_time from (values(1))", TABLE_NAME);
+
+    test("create table `%s/p3` as\n" +
+        "select timestamp '2017-01-05 05:46:11' as col_timestamp, date '2017-01-05' as col_date, time '05:46:11' as col_time from (values(1)) union all\n" +
+        "select timestamp '2017-01-06 06:17:59' as col_timestamp, date '2017-01-06' as col_date, time '06:17:59' as col_time from (values(1)) union all\n" +
+        "select timestamp '2017-01-06 06:17:59' as col_timestamp, date '2017-01-06' as col_date, time '06:17:59' as col_time from (values(1)) union all\n" +
+        "select cast(null as timestamp) as col_timestamp, cast(null as date) as col_date, cast(null as time) as col_time from (values(1))", TABLE_NAME);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    test("drop table if exists `%s`", TABLE_NAME);
+  }
+
+  @Test
+  public void testCastTimestampVarchar() throws Exception {
+    testParquetFilterPushDown("col_timestamp = '2017-01-05 05:46:11'", 1, 1);
+    testParquetFilterPushDown("col_timestamp = cast('2017-01-05 05:46:11' as varchar)", 1, 1);
+    testParquetFilterPushDown("col_timestamp = cast('2017-01-05 05:46:11' as timestamp)", 1, 1);
+    testParquetFilterPushDown("col_timestamp > '2017-01-02 00:00:00'", 7, 3);
+    testParquetFilterPushDown("col_timestamp between '2017-01-03 21:01:15' and '2017-01-06 05:46:11'", 3, 2);
+    testParquetFilterPushDown("col_timestamp between '2017-01-03' and '2017-01-06'", 4, 2);
+  }
+
+  @Test
+  public void testCastTimestampDate() throws Exception {
+    testParquetFilterPushDown("col_timestamp = date '2017-01-02'", 1, 1);
+    testParquetFilterPushDown("col_timestamp = cast(date '2017-01-02' as timestamp)", 1, 1);
+    testParquetFilterPushDown("col_timestamp > date '2017-01-02'", 7, 3);
+    testParquetFilterPushDown("col_timestamp between date '2017-01-03' and date '2017-01-06'", 4, 2);
+  }
+
+  @Test
+  public void testCastDateVarchar() throws Exception {
+    testParquetFilterPushDown("col_date = '2017-01-02'", 2, 1);
+    testParquetFilterPushDown("col_date = cast('2017-01-02' as varchar)", 2, 1);
+    testParquetFilterPushDown("col_date = cast('2017-01-02' as date)", 2, 1);
+    testParquetFilterPushDown("col_date > '2017-01-02'", 6, 2);
+    testParquetFilterPushDown("col_date between '2017-01-02' and '2017-01-04'", 5, 2);
+  }
+
+  @Test
+  public void testCastDateTimestamp() throws Exception {
+    testParquetFilterPushDown("col_date = timestamp '2017-01-02 00:00:00'", 2, 1);
+    testParquetFilterPushDown("col_date = cast(timestamp '2017-01-02 00:00:00' as date)", 2, 1);
+    testParquetFilterPushDown("col_date > timestamp '2017-01-02 21:01:15'", 6, 2);
+    testParquetFilterPushDown("col_date between timestamp '2017-01-03 08:50:00' and timestamp '2017-01-06 06:17:59'", 5, 2);
+  }
+
+  @Test
+  public void testCastTimeVarchar() throws Exception {
+    testParquetFilterPushDown("col_time = '00:00:00'", 2, 1);
+    testParquetFilterPushDown("col_time = cast('00:00:00' as varchar)", 2, 1);
+    testParquetFilterPushDown("col_time = cast('00:00:00' as time)", 2, 1);
+    testParquetFilterPushDown("col_time > '15:25:00'", 2, 2);
+    testParquetFilterPushDown("col_time between '08:00:00' and '23:00:00'", 4, 2);
+  }
+
+  @Test
+  public void testCastTimeTimestamp() throws Exception {
+    testParquetFilterPushDown("col_time = timestamp '2017-01-01 05:46:11'", 1, 2);
+    testParquetFilterPushDown("col_time = cast(timestamp '2017-01-01 05:46:11' as time)", 1, 2);
+    testParquetFilterPushDown("col_time = timestamp '2017-01-01 00:00:00'", 2, 1);
+    testParquetFilterPushDown("col_time > timestamp '2017-01-01 15:25:00'", 2, 2);
+    testParquetFilterPushDown("col_time between timestamp '2017-01-01 08:00:00' and timestamp '2017-01-01 23:00:00'", 4, 2);
+  }
+
+  @Test
+  public void testCastTimeDate() throws Exception {
+    testParquetFilterPushDown("col_time = date '2017-01-01'", 2, 1);
+    testParquetFilterPushDown("col_time = cast(date '2017-01-01' as time)", 2, 1);
+    testParquetFilterPushDown("col_time > date '2017-01-01'", 7, 3);
+    testParquetFilterPushDown("col_time between date '2017-01-01' and date '2017-01-02'", 2, 1);
+  }
+
+  private void testParquetFilterPushDown(String predicate, int expectedRowCount, int expectedFilesNumber) throws Exception {
+    String query = String.format("select * from `%s` where %s", TABLE_NAME, predicate);
+
+    int actualRowCount = testSql(query);
+    assertEquals("Expected and actual row count should match", expectedRowCount, actualRowCount);
+
+    String numFilesPattern = "numFiles=" + expectedFilesNumber;
+    testPlanMatchingPatterns(query, new String[] {numFilesPattern}, new String[] {});
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/3bc4e319/exec/java-exec/src/test/resources/parquetFilterPush/blnTbl/0_0_0.parquet
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/parquetFilterPush/blnTbl/0_0_0.parquet b/exec/java-exec/src/test/resources/parquetFilterPush/blnTbl/0_0_0.parquet
new file mode 100644
index 0000000..d3fb9ad
Binary files /dev/null and b/exec/java-exec/src/test/resources/parquetFilterPush/blnTbl/0_0_0.parquet differ

http://git-wip-us.apache.org/repos/asf/drill/blob/3bc4e319/exec/java-exec/src/test/resources/parquetFilterPush/blnTbl/0_0_1.parquet
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/parquetFilterPush/blnTbl/0_0_1.parquet b/exec/java-exec/src/test/resources/parquetFilterPush/blnTbl/0_0_1.parquet
new file mode 100644
index 0000000..bca65a2
Binary files /dev/null and b/exec/java-exec/src/test/resources/parquetFilterPush/blnTbl/0_0_1.parquet differ

http://git-wip-us.apache.org/repos/asf/drill/blob/3bc4e319/exec/java-exec/src/test/resources/parquetFilterPush/blnTbl/0_0_2.parquet
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/parquetFilterPush/blnTbl/0_0_2.parquet b/exec/java-exec/src/test/resources/parquetFilterPush/blnTbl/0_0_2.parquet
new file mode 100644
index 0000000..942947b
Binary files /dev/null and b/exec/java-exec/src/test/resources/parquetFilterPush/blnTbl/0_0_2.parquet differ

http://git-wip-us.apache.org/repos/asf/drill/blob/3bc4e319/exec/java-exec/src/test/resources/parquetFilterPush/blnTbl/0_0_3.parquet
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/parquetFilterPush/blnTbl/0_0_3.parquet b/exec/java-exec/src/test/resources/parquetFilterPush/blnTbl/0_0_3.parquet
new file mode 100644
index 0000000..48c717b
Binary files /dev/null and b/exec/java-exec/src/test/resources/parquetFilterPush/blnTbl/0_0_3.parquet differ


[06/17] drill git commit: DRILL-6190: Fix handling of packets longer than legally allowed

Posted by ar...@apache.org.
DRILL-6190: Fix handling of packets longer than legally allowed

closes #1133


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/57e5ab26
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/57e5ab26
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/57e5ab26

Branch: refs/heads/master
Commit: 57e5ab26659e1fa393f8963435de946e0da949be
Parents: 3bc4e31
Author: Ted Dunning <te...@gmail.com>
Authored: Wed Jan 10 16:52:53 2018 -0800
Committer: Arina Ielchiieva <ar...@gmail.com>
Committed: Sat Mar 3 19:47:40 2018 +0200

----------------------------------------------------------------------
 .../drill/exec/store/pcap/PcapRecordReader.java | 43 +++++++++++++-------
 .../store/pcap/decoder/PacketConstants.java     |  2 +-
 .../exec/store/pcap/decoder/PacketDecoder.java  | 15 ++++++-
 .../drill/exec/store/pcap/TestPcapDecoder.java  | 13 +++---
 .../exec/store/pcap/TestPcapRecordReader.java   |  5 ++-
 5 files changed, 52 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/57e5ab26/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapRecordReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapRecordReader.java
index a20e1de..26e1e65 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapRecordReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapRecordReader.java
@@ -56,8 +56,9 @@ import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.drill.exec.store.pcap.PcapFormatUtils.parseBytesToASCII;
 
 public class PcapRecordReader extends AbstractRecordReader {
-  private static final Logger logger = LoggerFactory.getLogger(PcapRecordReader.class);
+  static final int BUFFER_SIZE = 500_000;  // this should be relatively large relative to max packet
 
+  private static final Logger logger = LoggerFactory.getLogger(PcapRecordReader.class);
   private static final int BATCH_SIZE = 40_000;
 
   private OutputMutator output;
@@ -101,11 +102,10 @@ public class PcapRecordReader extends AbstractRecordReader {
   @Override
   public void setup(final OperatorContext context, final OutputMutator output) throws ExecutionSetupException {
     try {
-
       this.output = output;
-      this.buffer = new byte[100000];
       this.in = fs.open(pathToFile);
       this.decoder = new PacketDecoder(in);
+      this.buffer = new byte[BUFFER_SIZE + decoder.getMaxLength()];
       this.validBytes = in.read(buffer);
       this.projectedCols = getProjectedColsIfItNull();
       setColumns(projectedColumns);
@@ -197,20 +197,33 @@ public class PcapRecordReader extends AbstractRecordReader {
   private int parsePcapFilesAndPutItToTable() throws IOException {
     Packet packet = new Packet();
     int counter = 0;
-    while (offset < validBytes && counter != BATCH_SIZE) {
-
-      if (validBytes - offset < 9000) {
-        System.arraycopy(buffer, offset, buffer, 0, validBytes - offset);
-        validBytes = validBytes - offset;
-        offset = 0;
-
-        int n = in.read(buffer, validBytes, buffer.length - validBytes);
-        if (n > 0) {
-          validBytes += n;
+    while (offset < validBytes && counter < BATCH_SIZE) {
+      if (validBytes - offset < decoder.getMaxLength()) {
+        if (validBytes == buffer.length) {
+          // shift data and read more. This is the common case.
+          System.arraycopy(buffer, offset, buffer, 0, validBytes - offset);
+          validBytes = validBytes - offset;
+          offset = 0;
+
+          int n = in.read(buffer, validBytes, buffer.length - validBytes);
+          if (n > 0) {
+            validBytes += n;
+          }
+          logger.info("read {} bytes, at {} offset", n, validBytes);
+        } else {
+          // near the end of the file, we will just top up the buffer without shifting
+          int n = in.read(buffer, offset, buffer.length - offset);
+          if (n > 0) {
+            validBytes = validBytes + n;
+            logger.info("Topped up buffer with {} bytes to yield {}\n", n, validBytes);
+          }
         }
       }
-
-      offset = decoder.decodePacket(buffer, offset, packet);
+      int old = offset;
+      offset = decoder.decodePacket(buffer, offset, packet, decoder.getMaxLength(), validBytes);
+      if (offset > validBytes) {
+        logger.error("Invalid packet at offset {}", old);
+      }
 
       if (addDataToTable(packet, decoder.getNetwork(), counter)) {
         counter++;

http://git-wip-us.apache.org/repos/asf/drill/blob/57e5ab26/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/decoder/PacketConstants.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/decoder/PacketConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/decoder/PacketConstants.java
index 003f87e..2c87623 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/decoder/PacketConstants.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/decoder/PacketConstants.java
@@ -20,7 +20,7 @@ package org.apache.drill.exec.store.pcap.decoder;
 @SuppressWarnings("WeakerAccess")
 public final class PacketConstants {
 
-  public static final int PCAP_HEADER_SIZE = 4 * 4;
+  public static final int PCAP_HEADER_SIZE = 4 * 4;   // packet header, not file header
 
   public static final int TIMESTAMP_OFFSET = 0;
   public static final int TIMESTAMP_MICRO_OFFSET = 4;

http://git-wip-us.apache.org/repos/asf/drill/blob/57e5ab26/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/decoder/PacketDecoder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/decoder/PacketDecoder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/decoder/PacketDecoder.java
index 704c3fd..7460aaa 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/decoder/PacketDecoder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/decoder/PacketDecoder.java
@@ -18,6 +18,8 @@
 package org.apache.drill.exec.store.pcap.decoder;
 
 import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.io.InputStream;
@@ -58,6 +60,7 @@ public class PacketDecoder {
   private static final int PCAP_MAGIC_LITTLE_ENDIAN = 0xD4C3B2A1;
   private static final int PCAP_MAGIC_NUMBER = 0xA1B2C3D4;
 
+  private static final Logger logger = LoggerFactory.getLogger(PacketDecoder.class);
 
   private final int maxLength;
   private final int network;
@@ -89,8 +92,16 @@ public class PacketDecoder {
     network = getIntFileOrder(bigEndian, globalHeader, 20);
   }
 
-  public int decodePacket(final byte[] buffer, final int offset, Packet p) {
-    return p.decodePcap(buffer, offset, bigEndian, maxLength);
+  public final int getMaxLength() {
+    return maxLength;
+  }
+
+  public int decodePacket(final byte[] buffer, final int offset, Packet p, int maxPacket, int validBytes) {
+    int r = p.decodePcap(buffer, offset, bigEndian, Math.min(maxPacket, validBytes - offset));
+    if (r > validBytes) {
+      logger.error("Invalid packet at offset {}", offset);
+    }
+    return r;
   }
 
   public Packet packet() {

http://git-wip-us.apache.org/repos/asf/drill/blob/57e5ab26/exec/java-exec/src/test/java/org/apache/drill/exec/store/pcap/TestPcapDecoder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/pcap/TestPcapDecoder.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/pcap/TestPcapDecoder.java
index 6e58ccf..c8cedc1 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/pcap/TestPcapDecoder.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/pcap/TestPcapDecoder.java
@@ -58,12 +58,12 @@ public class TestPcapDecoder extends BaseTestQuery {
     int offset = 0;
 
 
-    byte[] buffer = new byte[100000];
+    byte[] buffer = new byte[PcapRecordReader.BUFFER_SIZE + pd.getMaxLength()];
     int validBytes = in.read(buffer);
     assertTrue(validBytes > 50);
 
-    offset = pd.decodePacket(buffer, offset, p);
-    offset = pd.decodePacket(buffer, offset, p);
+    offset = pd.decodePacket(buffer, offset, p, pd.getMaxLength(), validBytes);
+    offset = pd.decodePacket(buffer, offset, p, pd.getMaxLength(), validBytes);
     assertEquals(228, offset);
 
     assertEquals("FE:00:00:00:00:02", p.getEthernetDestination());
@@ -104,6 +104,7 @@ public class TestPcapDecoder extends BaseTestQuery {
   // the code from here down is useful in that it tests the assumptions that
   // the entire package is based on, but it doesn't really define tests.
   // As such, it can be run as a main class, but isn't supported as unit tests.
+
   /**
    * This tests the speed when creating an actual object for each packet.
    * <p>
@@ -163,7 +164,7 @@ public class TestPcapDecoder extends BaseTestQuery {
     PacketDecoder pd = new PacketDecoder(in);
     Packet p = pd.packet();
 
-    byte[] buffer = new byte[100000];
+    byte[] buffer = new byte[PcapRecordReader.BUFFER_SIZE + pd.getMaxLength()];
     int validBytes = in.read(buffer);
 
     int offset = 0;
@@ -176,7 +177,7 @@ public class TestPcapDecoder extends BaseTestQuery {
       // get new data and shift current data to beginning of buffer if there is any danger
       // of straddling the buffer end in the next packet
       // even with jumbo packets this should be enough space to guarantee parsing
-      if (validBytes - offset < 9000) {
+      if (validBytes - offset < pd.getMaxLength()) {
         System.arraycopy(buffer, 0, buffer, offset, validBytes - offset);
         validBytes = validBytes - offset;
         offset = 0;
@@ -188,7 +189,7 @@ public class TestPcapDecoder extends BaseTestQuery {
       }
 
       // decode the packet as it lies
-      offset = pd.decodePacket(buffer, offset, p);
+      offset = pd.decodePacket(buffer, offset, p, pd.getMaxLength(), validBytes);
       total += p.getPacketLength();
       allCount++;
       if (p.isTcpPacket()) {

http://git-wip-us.apache.org/repos/asf/drill/blob/57e5ab26/exec/java-exec/src/test/java/org/apache/drill/exec/store/pcap/TestPcapRecordReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/pcap/TestPcapRecordReader.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/pcap/TestPcapRecordReader.java
index 706694d..bb81469 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/pcap/TestPcapRecordReader.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/pcap/TestPcapRecordReader.java
@@ -48,7 +48,8 @@ public class TestPcapRecordReader extends BaseTestQuery {
 
   @Test
   public void testDistinctQuery() throws Exception {
-    runSQLVerifyCount("select distinct * from dfs.`store/pcap/tcp-1.pcap`", 1);
+    // omit data field from distinct count for now
+    runSQLVerifyCount("select distinct type, network, `timestamp`, src_ip, dst_ip, src_port, dst_port, src_mac_address, dst_mac_address, tcp_session, packet_length from dfs.`store/pcap/tcp-1.pcap`", 1);
   }
 
   private void runSQLVerifyCount(String sql, int expectedRowCount) throws Exception {
@@ -62,7 +63,7 @@ public class TestPcapRecordReader extends BaseTestQuery {
 
   private void printResultAndVerifyRowCount(List<QueryDataBatch> results,
                                             int expectedRowCount) throws SchemaChangeException {
-    setColumnWidth(25);
+    setColumnWidth(35);
     int rowCount = printResult(results);
     if (expectedRowCount != -1) {
       Assert.assertEquals(expectedRowCount, rowCount);


[09/17] drill git commit: DRILL-6188: Fix C++ client build on Centos7, OS X

Posted by ar...@apache.org.
DRILL-6188: Fix C++ client build on Centos7, OS X

closes #1132


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/a3b0bf59
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/a3b0bf59
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/a3b0bf59

Branch: refs/heads/master
Commit: a3b0bf5970b6d84b29ce843af1fd5cacf62a13bd
Parents: 4089422
Author: Parth Chandra <pa...@apache.org>
Authored: Tue Feb 27 17:31:51 2018 +0530
Committer: Arina Ielchiieva <ar...@gmail.com>
Committed: Sat Mar 3 19:47:45 2018 +0200

----------------------------------------------------------------------
 contrib/native/client/src/clientlib/drillClientImpl.hpp | 1 -
 contrib/native/client/src/clientlib/logger.hpp          | 1 +
 2 files changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/a3b0bf59/contrib/native/client/src/clientlib/drillClientImpl.hpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/clientlib/drillClientImpl.hpp b/contrib/native/client/src/clientlib/drillClientImpl.hpp
index 2e7623e..541c41e 100644
--- a/contrib/native/client/src/clientlib/drillClientImpl.hpp
+++ b/contrib/native/client/src/clientlib/drillClientImpl.hpp
@@ -182,7 +182,6 @@ class DrillClientBaseHandle: public DrillClientQueryHandle {
     virtual status_t notifyListener(ListenerValue v, DrillClientError* pErr);
 
     virtual void signalError(DrillClientError* pErr);
-    void setHasError(bool hasError) { m_bHasError = hasError; }
 
     private:
     Listener m_pApplicationListener;

http://git-wip-us.apache.org/repos/asf/drill/blob/a3b0bf59/contrib/native/client/src/clientlib/logger.hpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/clientlib/logger.hpp b/contrib/native/client/src/clientlib/logger.hpp
index 966e3a1..2eb98c0 100644
--- a/contrib/native/client/src/clientlib/logger.hpp
+++ b/contrib/native/client/src/clientlib/logger.hpp
@@ -19,6 +19,7 @@
 #ifndef __LOGGER_H
 #define __LOGGER_H
 
+#include <iostream>
 #include <sstream>
 #include <ostream>
 #include <iostream>


[11/17] drill git commit: DRILL-6197: Skip duplicate entry for OperatorStats

Posted by ar...@apache.org.
DRILL-6197: Skip duplicate entry for OperatorStats

org.apache.drill.exec.ops.FragmentStats should skip injecting the org.apache.drill.exec.ops.OperatorStats instance for these operators:
org.apache.drill.exec.proto.beans.CoreOperatorType.SCREEN
org.apache.drill.exec.proto.beans.CoreOperatorType.SINGLE_SENDER
org.apache.drill.exec.proto.beans.CoreOperatorType.BROADCAST_SENDER
org.apache.drill.exec.proto.beans.CoreOperatorType.HASH_PARTITION_SENDER

closes #1141


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/6af651fc
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/6af651fc
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/6af651fc

Branch: refs/heads/master
Commit: 6af651fcde8059dbf557a1f2f897557425fb950d
Parents: 161a046
Author: Kunal Khatua <kk...@maprtech.com>
Authored: Wed Feb 28 14:07:27 2018 -0800
Committer: Arina Ielchiieva <ar...@gmail.com>
Committed: Sat Mar 3 19:47:48 2018 +0200

----------------------------------------------------------------------
 .../apache/drill/exec/ops/FragmentStats.java    | 20 +++++++++++---------
 .../drill/exec/physical/impl/BaseRootExec.java  | 16 ++++++++--------
 2 files changed, 19 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/6af651fc/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentStats.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentStats.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentStats.java
index a173073..cdad6e4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentStats.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentStats.java
@@ -17,21 +17,22 @@
  */
 package org.apache.drill.exec.ops;
 
-import java.util.List;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Map.Entry;
 
+import org.apache.commons.lang3.tuple.ImmutablePair;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile;
 
-import com.google.common.collect.Lists;
-
 /**
  * Holds statistics of a particular (minor) fragment.
  */
 public class FragmentStats {
 //  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentStats.class);
 
-  private List<OperatorStats> operators = Lists.newArrayList();
+  private Map<ImmutablePair<Integer, Integer>, OperatorStats> operators = new LinkedHashMap<>();
   private final long startTime;
   private final DrillbitEndpoint endpoint;
   private final BufferAllocator allocator;
@@ -47,8 +48,8 @@ public class FragmentStats {
     prfB.setMaxMemoryUsed(allocator.getPeakMemoryAllocation());
     prfB.setEndTime(System.currentTimeMillis());
     prfB.setEndpoint(endpoint);
-    for(OperatorStats o : operators){
-      prfB.addOperatorProfile(o.getProfile());
+    for(Entry<ImmutablePair<Integer, Integer>, OperatorStats> o : operators.entrySet()){
+      prfB.addOperatorProfile(o.getValue().getProfile());
     }
   }
 
@@ -62,13 +63,14 @@ public class FragmentStats {
   public OperatorStats newOperatorStats(final OpProfileDef profileDef, final BufferAllocator allocator) {
     final OperatorStats stats = new OperatorStats(profileDef, allocator);
     if(profileDef.operatorType != -1) {
-      operators.add(stats);
+      @SuppressWarnings("unused")
+      OperatorStats existingStatsHolder = addOperatorStats(stats);
     }
     return stats;
   }
 
-  public void addOperatorStats(OperatorStats stats) {
-    operators.add(stats);
+  public OperatorStats addOperatorStats(OperatorStats stats) {
+    return operators.put(new ImmutablePair<>(stats.operatorId, stats.operatorType), stats);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/6af651fc/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/BaseRootExec.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/BaseRootExec.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/BaseRootExec.java
index 82887ec..bf52d04 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/BaseRootExec.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/BaseRootExec.java
@@ -43,20 +43,20 @@ public abstract class BaseRootExec implements RootExec {
   private List<CloseableRecordBatch> operators;
 
   public BaseRootExec(final RootFragmentContext fragmentContext, final PhysicalOperator config) throws OutOfMemoryException {
-    this.oContext = fragmentContext.newOperatorContext(config, stats);
-    stats = new OperatorStats(new OpProfileDef(config.getOperatorId(),
-        config.getOperatorType(), OperatorUtilities.getChildCount(config)),
-        oContext.getAllocator());
-    fragmentContext.getStats().addOperatorStats(this.stats);
-    this.fragmentContext = fragmentContext;
+    this(fragmentContext, null, config);
   }
 
   public BaseRootExec(final RootFragmentContext fragmentContext, final OperatorContext oContext,
                       final PhysicalOperator config) throws OutOfMemoryException {
-    this.oContext = oContext;
+    if (oContext == null) {
+      this.oContext = fragmentContext.newOperatorContext(config, stats);
+    } else {
+      this.oContext = oContext;
+    }
+    //Creating new stat for appending to list
     stats = new OperatorStats(new OpProfileDef(config.getOperatorId(),
         config.getOperatorType(), OperatorUtilities.getChildCount(config)),
-      oContext.getAllocator());
+      this.oContext.getAllocator());
     fragmentContext.getStats().addOperatorStats(this.stats);
     this.fragmentContext = fragmentContext;
   }


[02/17] drill git commit: DRILL-6205: Reduce memory consumption of testFlattenUpperLimit test

Posted by ar...@apache.org.
DRILL-6205: Reduce memory consumption of testFlattenUpperLimit test

closes #1147


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/41125360
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/41125360
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/41125360

Branch: refs/heads/master
Commit: 411253608344343f09f75e15af61e79358da6eb1
Parents: 49faae0
Author: Padma Penumarthy <pp...@yahoo.com>
Authored: Fri Mar 2 11:13:59 2018 -0800
Committer: Arina Ielchiieva <ar...@gmail.com>
Committed: Sat Mar 3 19:47:33 2018 +0200

----------------------------------------------------------------------
 .../exec/physical/unit/TestOutputBatchSize.java   | 18 +++++++++---------
 1 file changed, 9 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/41125360/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestOutputBatchSize.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestOutputBatchSize.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestOutputBatchSize.java
index 99af4c2..741164f 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestOutputBatchSize.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestOutputBatchSize.java
@@ -656,12 +656,12 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
 
     batchString.append("[");
 
-    numRows = 1000;
+    numRows = 100;
 
     for (int i = 0; i < numRows; i++) {
-      batchString.append("{\"a\": 5, " + "\"b\" : " + "\"" + wideString + "\"," + "\"c\":" + flattenElement + "},");
+      batchString.append("{\"a\": 5, "  + "\"c\":" + flattenElement + "},");
     }
-    batchString.append("{\"a\": 5, " + "\"b\" : " + "\"" + wideString + "\"," + "\"c\":" + flattenElement + "}");
+    batchString.append("{\"a\": 5, " + "\"c\":" + flattenElement + "}");
     batchString.append("]");
     inputJsonBatches.add(batchString.toString());
 
@@ -674,18 +674,18 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
     expectedBatchString.append("[");
     for (int i = 0; i < numRows; i++) {
       for (int j = 0; j < 1000; j++) {
-        expectedBatchString.append("{\"a\": 5, " + "\"b\" : " + "\"" + wideString + "\"," + "\"c\" :");
+        expectedBatchString.append("{\"a\": 5, "  + "\"c\" :");
         expectedBatchString.append(j);
         expectedBatchString.append("},");
       }
     }
     for (int j = 0; j < 999; j++) {
-      expectedBatchString.append("{\"a\": 5, " + "\"b\" : " + "\"" + wideString + "\"," + "\"c\" :");
+      expectedBatchString.append("{\"a\": 5, "  + "\"c\" :");
       expectedBatchString.append(j);
       expectedBatchString.append("},");
     }
 
-    expectedBatchString.append("{\"a\": 5, " + "\"b\" : " + "\"" + wideString + "\"," + "\"c\" :");
+    expectedBatchString.append("{\"a\": 5, "  + "\"c\" :");
     expectedBatchString.append(1000);
     expectedBatchString.append("}");
 
@@ -701,13 +701,13 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
     OperatorTestBuilder opTestBuilder = opTestBuilder()
       .physicalOperator(flatten)
       .inputDataStreamJson(inputJsonBatches)
-      .baselineColumns("a", "b", "c")
-      .expectedNumBatches(16) // verify number of batches
+      .baselineColumns("a", "c")
+      .expectedNumBatches(2) // verify number of batches
       .expectedBatchSize(totalSize / 2);  // verify batch size.
 
     for (long i = 0; i < numRows + 1; i++) {
       for (long j = 0; j < 1001; j++) {
-        opTestBuilder.baselineValues(5l, wideString, j);
+        opTestBuilder.baselineValues(5l, j);
       }
     }
 


[03/17] drill git commit: DRILL-6195: Quering Hive non-partitioned transactional tables via Drill

Posted by ar...@apache.org.
DRILL-6195: Quering Hive non-partitioned transactional tables via Drill

closes #1140


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/6bf04ecc
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/6bf04ecc
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/6bf04ecc

Branch: refs/heads/master
Commit: 6bf04eccf4ab756fa370f800f2d35d2443bac518
Parents: 4112536
Author: Vitalii Diravka <vi...@gmail.com>
Authored: Wed Feb 28 14:16:17 2018 +0000
Committer: Arina Ielchiieva <ar...@gmail.com>
Committed: Sat Mar 3 19:47:35 2018 +0200

----------------------------------------------------------------------
 .../apache/drill/exec/store/hive/readers/HiveAbstractReader.java  | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/6bf04ecc/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/readers/HiveAbstractReader.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/readers/HiveAbstractReader.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/readers/HiveAbstractReader.java
index b814866..a922b4c 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/readers/HiveAbstractReader.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/readers/HiveAbstractReader.java
@@ -153,7 +153,7 @@ public abstract class HiveAbstractReader extends AbstractRecordReader {
         finalOI = (StructObjectInspector)ObjectInspectorConverters.getConvertedOI(partitionOI, tableOI);
         partTblObjectInspectorConverter = ObjectInspectorConverters.getConverter(partitionOI, finalOI);
         job.setInputFormat(HiveUtilities.getInputFormatClass(job, partition.getSd(), table));
-        HiveUtilities.verifyAndAddTransactionalProperties(job, table.getSd());
+        HiveUtilities.verifyAndAddTransactionalProperties(job, partition.getSd());
       } else {
         // For non-partitioned tables, there is no need to create converter as there are no schema changes expected.
         partitionDeserializer = tableDeserializer;
@@ -161,6 +161,7 @@ public abstract class HiveAbstractReader extends AbstractRecordReader {
         partTblObjectInspectorConverter = null;
         finalOI = tableOI;
         job.setInputFormat(HiveUtilities.getInputFormatClass(job, table.getSd(), table));
+        HiveUtilities.verifyAndAddTransactionalProperties(job, table.getSd());
       }
 
       if (logger.isTraceEnabled()) {


[10/17] drill git commit: DRILL-6185: Fixed error while displaying system profiles via the Web-UI

Posted by ar...@apache.org.
DRILL-6185: Fixed error while displaying system profiles via the Web-UI

The bug lies in the absence of a text plan for profiles related to system queries:
e.g.
{code:sql}
use dfs.tmp;
show tables;
alter session reset all;
{code}
This addresses that by ensuring an empty string is substituted and the tokenization is done correctly.

closes #1137


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/161a0468
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/161a0468
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/161a0468

Branch: refs/heads/master
Commit: 161a046822ca6676e0c62dcbdf38abcc6c08b1b6
Parents: a3b0bf5
Author: Kunal Khatua <kk...@maprtech.com>
Authored: Tue Feb 27 12:10:06 2018 -0800
Committer: Arina Ielchiieva <ar...@gmail.com>
Committed: Sat Mar 3 19:47:46 2018 +0200

----------------------------------------------------------------------
 .../server/rest/profile/OperatorWrapper.java    |  3 ++-
 .../server/rest/profile/ProfileWrapper.java     | 20 ++++++++++++++------
 2 files changed, 16 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/161a0468/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/OperatorWrapper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/OperatorWrapper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/OperatorWrapper.java
index 0df062f..afccbb7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/OperatorWrapper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/OperatorWrapper.java
@@ -25,7 +25,6 @@ import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
 
-import com.google.common.base.Preconditions;
 import org.apache.commons.lang3.tuple.ImmutablePair;
 import org.apache.drill.exec.ops.OperatorMetricRegistry;
 import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
@@ -33,6 +32,8 @@ import org.apache.drill.exec.proto.UserBitShared.MetricValue;
 import org.apache.drill.exec.proto.UserBitShared.OperatorProfile;
 import org.apache.drill.exec.proto.UserBitShared.StreamProfile;
 
+import com.google.common.base.Preconditions;
+
 /**
  * Wrapper class for profiles of ALL operator instances of the same operator type within a major fragment.
  */

http://git-wip-us.apache.org/repos/asf/drill/blob/161a0468/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileWrapper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileWrapper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileWrapper.java
index 2c964da..a618f7e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileWrapper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileWrapper.java
@@ -17,15 +17,14 @@
  */
 package org.apache.drill.exec.server.rest.profile;
 
+import static com.fasterxml.jackson.databind.SerializationFeature.INDENT_OUTPUT;
+
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.common.base.CaseFormat;
-import com.google.common.collect.Maps;
 import org.apache.commons.lang3.tuple.ImmutablePair;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
@@ -39,7 +38,9 @@ import org.apache.drill.exec.server.options.OptionList;
 import org.apache.drill.exec.server.options.OptionValue;
 import org.apache.drill.exec.server.rest.WebServer;
 
-import static com.fasterxml.jackson.databind.SerializationFeature.INDENT_OUTPUT;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.CaseFormat;
+import com.google.common.collect.Maps;
 
 /**
  * Wrapper class for a {@link #profile query profile}, so it to be presented through web UI.
@@ -64,7 +65,8 @@ public class ProfileWrapper {
     this.profile = profile;
     this.id = QueryIdHelper.getQueryId(profile.getId());
     //Generating Operator Name map (DRILL-6140)
-    generateOpMap(profile.getPlan());
+    String profileTextPlan = profile.hasPlan() ? profile.getPlan() : "" ;
+    generateOpMap(profileTextPlan);
 
     final List<FragmentWrapper> fragmentProfiles = new ArrayList<>();
 
@@ -329,11 +331,17 @@ public class ProfileWrapper {
 
   //Generates operator names inferred from physical plan
   private void generateOpMap(String plan) {
-    this.physicalOperatorMap = new HashMap<String,String>();
+    this.physicalOperatorMap = new HashMap<>();
+    if (plan.isEmpty()) {
+      return;
+    }
     //[e.g ] operatorLine = "01-03 Flatten(flattenField=[$1]) : rowType = RecordType(ANY rfsSpecCode, ..."
     String[] operatorLine = plan.split("\\n");
     for (String line : operatorLine) {
       String[] lineToken = line.split("\\s+", 3);
+      if (lineToken.length < 2) {
+        continue; //Skip due to possible invalid entry
+      }
       //[e.g ] operatorPath = "01-xx-03"
       String operatorPath = lineToken[0].trim().replaceFirst("-", "-xx-"); //Required format for lookup
       //[e.g ] extractedOperatorName = "FLATTEN"


[15/17] drill git commit: DRILL-6114: Metadata revisions

Posted by ar...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/VariantMetadata.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/VariantMetadata.java b/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/VariantMetadata.java
new file mode 100644
index 0000000..d01f292
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/VariantMetadata.java
@@ -0,0 +1,177 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.record.metadata;
+
+import java.util.Collection;
+
+import org.apache.drill.common.types.TypeProtos.MinorType;
+
+/**
+ * Describes the contents of a list or union field. Such fields are,
+ * in effect, a map from minor type to vector, represented here as
+ * a map from minor type to column metadata. The child columns used here
+ * are a useful fiction. The column name is made up to be the same as
+ * the name of the type.
+ * <p>
+ * In Drill, a union and a list are related, but distinct. In metadata,
+ * a union is an optional variant while a list is a variant array.
+ * This makes the representation simpler
+ * and should be a good-enough approximation of reality.
+ * <p>
+ * Variants can contain three kinds of children:
+ * <ul>
+ * <li>Nullable (optional) scalar vectors.</li>
+ * <li>Non-nullable (required) map.</li>
+ * <li>Nullable (optional) list.</li>
+ * </ul>
+ * <p>
+ * A union cannot contain a repeated vector. Instead, the
+ * union can contain a list. Note also that maps can never be optional,
+ * so they are required in the union, even though the map is, in effect,
+ * optional (the map is effectively null if it is not used for a give
+ * row.) Yes, this is confusing, but it is how the vectors are
+ * implemented (for now.)
+ * <p>
+ * A list type is modeled here as a repeated union type. This is not
+ * entirely accurate, but it is another useful fiction. (In actual
+ * implementation, a list is either a single type, or a array of
+ * unions. This detail is abstracted away here.)
+ * <p>
+ * In vector implementation, unions declare their member types, but
+ * lists don't. Here, both types declare their member types. (Another
+ * useful fiction.)
+ * <p>
+ * A union or list can contain a map. Maps have structure. To support this,
+ * the metadata allows adding a map column that contains the map structure.
+ * Such metadata exist only in this system; it is not easily accessible in
+ * the vector implementation.
+ * <p>
+ * A union or list can contain a list (though not a union.) As described
+ * here, lists can have structure, and so, like maps, can be built using
+ * a column that provides that structure.
+ * <p>
+ * Note that the Drill {@link MinorType#UNION UNION} and
+ * {@link MinorType#LIST LIST} implementations are considered experimental
+ * and are not generally enabled. As a result, this metadata schema must
+ * also be considered experimental and subject to change.
+ */
+
+public interface VariantMetadata {
+
+  /**
+   * Add any supported type to the variant.
+   * <p>
+   * At present, the union
+   * vector does not support the decimal types. This class does not
+   * reject such types; but they will cause a runtime exception when
+   * code asks the union vector for these types.
+   *
+   * @param type type to add
+   * @return the "virtual" column for that type
+   * @throws IllegalArgumentException if the type has already been
+   * added
+   */
+
+  ColumnMetadata addType(MinorType type);
+
+  /**
+   * Add a column for any supported type to the variant.
+   * Use this to add structure to a list or map member.
+   *
+   * @param col column to add. The column must have the correct
+   * mode. The column's type is used as the type key
+   * @throws IllegalArgumentException if the type has already been
+   * added, or if the mode is wrong
+   */
+
+  void addType(ColumnMetadata col);
+
+  /**
+   * Returns the number of types in the variant.
+   *
+   * @return the number of types in the variant
+   */
+
+  int size();
+
+  /**
+   * Determine if the given type is a member of the variant.
+   *
+   * @param type type to check
+   * @return <tt>true</tt> if the type is a member,
+   * <tt>false</tt> if not
+   */
+
+  boolean hasType(MinorType type);
+
+  /**
+   * Returns the list of types which are members of this variant.
+   *
+   * @return the list of types
+   */
+
+  Collection<MinorType> types();
+
+  Collection<ColumnMetadata> members();
+
+  /**
+   * Retrieve the virtual column for a given type.
+   *
+   * @param type the type key
+   * @return the virtual column, or <tt>null</tt> if the type
+   * is not a member of the variant
+   */
+
+  ColumnMetadata member(MinorType type);
+
+  /**
+   * Return the column that defines this variant structure
+   *
+   * @return the column that returns this variant structure
+   * from its {@link ColumnMetadata#variantSchema() variantSchema()}
+   * method
+   */
+
+  ColumnMetadata parent();
+
+  /**
+   * A list is defined as a list of variants at the metadata layer.
+   * But, in implementation, a list will do special processing if the
+   * variant (union) contains only one type.
+   *
+   * @return <tt>true</tt> if this variant contains only one type,
+   * </tt>false</tt> if the variant contains 0, 2 or more types
+   */
+
+  boolean isSingleType();
+
+  /**
+   * Lists are odd creatures: they contain a union if they have more
+   * than one subtype, but are like a nullable repeated type if they
+   * contain only one type. This method returns the type of the array:
+   * either the single type (if {@link #isSingleType()} is <tt>true</tt>)
+   * or a reference to the synthetic union column nested inside the
+   * list.
+   * @return the metadata for the implicit column within the list
+   */
+
+  ColumnMetadata listSubtype();
+
+  void becomeSimple();
+  boolean isSimple();
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ObjectWriter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ObjectWriter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ObjectWriter.java
index a49b0d8..113778f 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ObjectWriter.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ObjectWriter.java
@@ -17,7 +17,7 @@
  */
 package org.apache.drill.exec.vector.accessor;
 
-import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.apache.drill.exec.vector.accessor.ScalarWriter.ColumnWriterListener;
 import org.apache.drill.exec.vector.accessor.TupleWriter.TupleWriterListener;
 

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/TupleReader.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/TupleReader.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/TupleReader.java
index 908d6a0..8d691c3 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/TupleReader.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/TupleReader.java
@@ -17,7 +17,7 @@
  */
 package org.apache.drill.exec.vector.accessor;
 
-import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 
 /**
  * Interface for reading from tuples (rows or maps). Provides

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/TupleWriter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/TupleWriter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/TupleWriter.java
index 056c9b3..0a52283 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/TupleWriter.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/TupleWriter.java
@@ -17,9 +17,9 @@
  */
 package org.apache.drill.exec.vector.accessor;
 
-import org.apache.drill.exec.record.ColumnMetadata;
 import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 
 /**
  * Writer for a tuple. A tuple is composed of columns with a fixed order and

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/AbstractTupleReader.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/AbstractTupleReader.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/AbstractTupleReader.java
index afa0cb7..0429f3e 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/AbstractTupleReader.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/AbstractTupleReader.java
@@ -20,7 +20,7 @@ package org.apache.drill.exec.vector.accessor.reader;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.vector.accessor.ArrayReader;
 import org.apache.drill.exec.vector.accessor.ColumnReaderIndex;
 import org.apache.drill.exec.vector.accessor.ObjectReader;

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/MapReader.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/MapReader.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/MapReader.java
index 66bc067..900e0a7 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/MapReader.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/MapReader.java
@@ -19,7 +19,7 @@ package org.apache.drill.exec.vector.accessor.reader;
 
 import java.util.List;
 
-import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
 
 /**
  * Reader for a Drill Map type. Maps are actually tuples, just like rows.

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractArrayWriter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractArrayWriter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractArrayWriter.java
index e6e29b4..58cda57 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractArrayWriter.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractArrayWriter.java
@@ -17,7 +17,7 @@
  */
 package org.apache.drill.exec.vector.accessor.writer;
 
-import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.apache.drill.exec.vector.UInt4Vector;
 import org.apache.drill.exec.vector.accessor.ArrayWriter;
 import org.apache.drill.exec.vector.accessor.ColumnWriterIndex;

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractObjectWriter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractObjectWriter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractObjectWriter.java
index a8f1c64..15807cb 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractObjectWriter.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractObjectWriter.java
@@ -17,7 +17,7 @@
  */
 package org.apache.drill.exec.vector.accessor.writer;
 
-import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.apache.drill.exec.vector.accessor.ArrayWriter;
 import org.apache.drill.exec.vector.accessor.ObjectWriter;
 import org.apache.drill.exec.vector.accessor.ScalarWriter;

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractScalarWriter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractScalarWriter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractScalarWriter.java
index c02e2d9..b6a85d7 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractScalarWriter.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractScalarWriter.java
@@ -19,7 +19,7 @@ package org.apache.drill.exec.vector.accessor.writer;
 
 import java.math.BigDecimal;
 
-import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.apache.drill.exec.vector.BaseDataValueVector;
 import org.apache.drill.exec.vector.accessor.ObjectType;
 import org.apache.drill.exec.vector.accessor.ScalarWriter;

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractTupleWriter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractTupleWriter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractTupleWriter.java
index 1fd12f2..938f867 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractTupleWriter.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractTupleWriter.java
@@ -20,9 +20,9 @@ package org.apache.drill.exec.vector.accessor.writer;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.drill.exec.record.ColumnMetadata;
 import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.vector.accessor.ArrayWriter;
 import org.apache.drill.exec.vector.accessor.ColumnWriterIndex;
 import org.apache.drill.exec.vector.accessor.ObjectType;

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/ColumnWriterFactory.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/ColumnWriterFactory.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/ColumnWriterFactory.java
index 5a1187a..30811bb 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/ColumnWriterFactory.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/ColumnWriterFactory.java
@@ -24,7 +24,7 @@ import java.util.List;
 
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.TypeProtos.MinorType;
-import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.apache.drill.exec.vector.NullableVector;
 import org.apache.drill.exec.vector.UInt4Vector;
 import org.apache.drill.exec.vector.ValueVector;

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/MapWriter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/MapWriter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/MapWriter.java
index 8aec301..af8daba 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/MapWriter.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/MapWriter.java
@@ -19,7 +19,7 @@ package org.apache.drill.exec.vector.accessor.writer;
 
 import java.util.List;
 
-import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.apache.drill.exec.vector.accessor.ColumnWriterIndex;
 import org.apache.drill.exec.vector.complex.MapVector;
 

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/NullableScalarWriter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/NullableScalarWriter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/NullableScalarWriter.java
index 2068304..c5f7982 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/NullableScalarWriter.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/NullableScalarWriter.java
@@ -19,7 +19,7 @@ package org.apache.drill.exec.vector.accessor.writer;
 
 import java.math.BigDecimal;
 
-import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.apache.drill.exec.vector.BaseDataValueVector;
 import org.apache.drill.exec.vector.NullableVector;
 import org.apache.drill.exec.vector.accessor.ColumnAccessors.UInt1ColumnWriter;

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/ScalarArrayWriter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/ScalarArrayWriter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/ScalarArrayWriter.java
index 95f8f29..8382ad1 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/ScalarArrayWriter.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/ScalarArrayWriter.java
@@ -19,7 +19,7 @@ package org.apache.drill.exec.vector.accessor.writer;
 
 import java.math.BigDecimal;
 
-import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.apache.drill.exec.vector.accessor.ColumnWriterIndex;
 import org.apache.drill.exec.vector.accessor.ScalarWriter.ColumnWriterListener;
 import org.apache.drill.exec.vector.accessor.writer.AbstractArrayWriter.BaseArrayWriter;


[17/17] drill git commit: DRILL-6114: Metadata revisions

Posted by ar...@apache.org.
DRILL-6114: Metadata revisions

Support for union vectors, list vectors, repeated list vectors. Refactored metadata classes.

closes #1112


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/cf2478f7
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/cf2478f7
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/cf2478f7

Branch: refs/heads/master
Commit: cf2478f7a48e66cbb0a7f29750c8a4360a271e9b
Parents: 69a5f3a
Author: Paul Rogers <pr...@cloudera.com>
Authored: Mon Feb 5 20:18:18 2018 -0800
Committer: Arina Ielchiieva <ar...@gmail.com>
Committed: Sat Mar 3 19:47:55 2018 +0200

----------------------------------------------------------------------
 .../exec/physical/rowSet/ResultSetLoader.java   |   2 +-
 .../exec/physical/rowSet/impl/ColumnState.java  |   2 +-
 .../rowSet/impl/NullableVectorState.java        |   2 +-
 .../physical/rowSet/impl/OptionBuilder.java     |   2 +-
 .../rowSet/impl/RepeatedVectorState.java        |   2 +-
 .../rowSet/impl/ResultSetLoaderImpl.java        |   2 +-
 .../physical/rowSet/impl/RowSetLoaderImpl.java  |   2 +-
 .../physical/rowSet/impl/SingleVectorState.java |   2 +-
 .../exec/physical/rowSet/impl/TupleState.java   |  11 +-
 .../rowSet/impl/VectorContainerBuilder.java     |   6 +-
 .../physical/rowSet/model/BaseTupleModel.java   |   8 +-
 .../physical/rowSet/model/MetadataProvider.java |   6 +-
 .../physical/rowSet/model/SchemaInference.java  |  15 +-
 .../exec/physical/rowSet/model/TupleModel.java  |   6 +-
 .../model/single/BuildVectorsFromMetadata.java  |   4 +-
 .../rowSet/model/single/VectorAllocator.java    |   4 +-
 .../apache/drill/exec/record/TupleSchema.java   | 534 -------------------
 .../record/metadata/AbstractColumnMetadata.java | 206 +++++++
 .../exec/record/metadata/MapColumnMetadata.java | 126 +++++
 .../exec/record/metadata/MetadataUtils.java     | 165 ++++++
 .../metadata/PrimitiveColumnMetadata.java       | 131 +++++
 .../metadata/RepeatedListColumnMetadata.java    | 100 ++++
 .../drill/exec/record/metadata/TupleSchema.java | 196 +++++++
 .../record/metadata/VariantColumnMetadata.java  | 136 +++++
 .../exec/record/metadata/VariantSchema.java     | 210 ++++++++
 .../physical/impl/xsort/TestExternalSort.java   |  20 +-
 .../impl/xsort/managed/SortTestUtilities.java   |   2 +-
 .../impl/TestResultSetLoaderMapArray.java       |   2 +-
 .../rowSet/impl/TestResultSetLoaderMaps.java    |   2 +-
 .../impl/TestResultSetLoaderOmittedValues.java  |   2 +-
 .../impl/TestResultSetLoaderOverflow.java       |   2 +-
 .../impl/TestResultSetLoaderProjection.java     |   4 +-
 .../impl/TestResultSetLoaderProtocol.java       |  57 +-
 .../rowSet/impl/TestResultSetLoaderTorture.java |   2 +-
 .../drill/exec/physical/unit/TestMiniPlan.java  |  10 +-
 .../drill/exec/record/TestTupleSchema.java      | 264 +++++++--
 .../exec/store/easy/text/compliant/TestCsv.java |   3 +-
 .../org/apache/drill/test/DrillTestWrapper.java |   6 +-
 .../org/apache/drill/test/OperatorFixture.java  |  17 +-
 .../drill/test/rowSet/AbstractRowSet.java       |   2 +-
 .../drill/test/rowSet/AbstractSingleRowSet.java |   2 +-
 .../apache/drill/test/rowSet/DirectRowSet.java  |   8 +-
 .../drill/test/rowSet/HyperRowSetImpl.java      |   2 +-
 .../org/apache/drill/test/rowSet/RowSet.java    |   2 +-
 .../apache/drill/test/rowSet/RowSetBuilder.java |   6 +-
 .../apache/drill/test/rowSet/RowSetPrinter.java |   4 +-
 .../drill/test/rowSet/RowSetReaderImpl.java     |   2 +-
 .../drill/test/rowSet/RowSetWriterImpl.java     |   2 +-
 .../apache/drill/test/rowSet/SchemaBuilder.java |  11 +-
 .../drill/test/rowSet/test/DummyWriterTest.java |   2 +-
 .../drill/test/rowSet/test/PerformanceTool.java |   8 +-
 .../drill/test/rowSet/test/RowSetTest.java      |   2 +-
 .../drill/test/rowSet/test/TestFillEmpties.java |   2 +-
 .../drill/exec/record/ColumnMetadata.java       | 114 ----
 .../apache/drill/exec/record/TupleMetadata.java |  88 ---
 .../drill/exec/record/TupleNameSpace.java       |  89 ----
 .../exec/record/metadata/ColumnMetadata.java    | 196 +++++++
 .../exec/record/metadata/ProjectionType.java    |  27 +
 .../exec/record/metadata/TupleMetadata.java     |  90 ++++
 .../exec/record/metadata/TupleNameSpace.java    |  89 ++++
 .../exec/record/metadata/VariantMetadata.java   | 177 ++++++
 .../exec/vector/accessor/ObjectWriter.java      |   2 +-
 .../drill/exec/vector/accessor/TupleReader.java |   2 +-
 .../drill/exec/vector/accessor/TupleWriter.java |   4 +-
 .../accessor/reader/AbstractTupleReader.java    |   2 +-
 .../exec/vector/accessor/reader/MapReader.java  |   2 +-
 .../accessor/writer/AbstractArrayWriter.java    |   2 +-
 .../accessor/writer/AbstractObjectWriter.java   |   2 +-
 .../accessor/writer/AbstractScalarWriter.java   |   2 +-
 .../accessor/writer/AbstractTupleWriter.java    |   4 +-
 .../accessor/writer/ColumnWriterFactory.java    |   2 +-
 .../exec/vector/accessor/writer/MapWriter.java  |   2 +-
 .../accessor/writer/NullableScalarWriter.java   |   2 +-
 .../accessor/writer/ScalarArrayWriter.java      |   2 +-
 74 files changed, 2221 insertions(+), 1007 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/ResultSetLoader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/ResultSetLoader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/ResultSetLoader.java
index a4b260b..0bfb948 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/ResultSetLoader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/ResultSetLoader.java
@@ -17,8 +17,8 @@
  */
 package org.apache.drill.exec.physical.rowSet;
 
-import org.apache.drill.exec.record.TupleMetadata;
 import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.vector.BaseValueVector;
 import org.apache.drill.exec.vector.complex.impl.VectorContainerWriter;
 

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/ColumnState.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/ColumnState.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/ColumnState.java
index f3626d9..33d3ffe 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/ColumnState.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/ColumnState.java
@@ -22,7 +22,7 @@ import java.util.ArrayList;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.physical.rowSet.impl.SingleVectorState.OffsetVectorState;
 import org.apache.drill.exec.physical.rowSet.impl.TupleState.MapState;
-import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.apache.drill.exec.vector.UInt4Vector;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.accessor.impl.HierarchicalFormatter;

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/NullableVectorState.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/NullableVectorState.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/NullableVectorState.java
index bf91032..7e8080f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/NullableVectorState.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/NullableVectorState.java
@@ -18,7 +18,7 @@
 package org.apache.drill.exec.physical.rowSet.impl;
 
 import org.apache.drill.exec.physical.rowSet.impl.SingleVectorState.ValuesVectorState;
-import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.apache.drill.exec.vector.FixedWidthVector;
 import org.apache.drill.exec.vector.NullableVector;
 import org.apache.drill.exec.vector.ValueVector;

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/OptionBuilder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/OptionBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/OptionBuilder.java
index a743052..fa62a05 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/OptionBuilder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/OptionBuilder.java
@@ -22,7 +22,7 @@ import java.util.Collection;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.physical.rowSet.ResultVectorCache;
 import org.apache.drill.exec.physical.rowSet.impl.ResultSetLoaderImpl.ResultSetOptions;
-import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.vector.BaseValueVector;
 import org.apache.drill.exec.vector.ValueVector;
 

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/RepeatedVectorState.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/RepeatedVectorState.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/RepeatedVectorState.java
index 98b6beb..9bd1ef2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/RepeatedVectorState.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/RepeatedVectorState.java
@@ -19,7 +19,7 @@ package org.apache.drill.exec.physical.rowSet.impl;
 
 import org.apache.drill.exec.physical.rowSet.impl.SingleVectorState.OffsetVectorState;
 import org.apache.drill.exec.physical.rowSet.impl.SingleVectorState.ValuesVectorState;
-import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.accessor.impl.HierarchicalFormatter;
 import org.apache.drill.exec.vector.accessor.writer.AbstractArrayWriter;

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/ResultSetLoaderImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/ResultSetLoaderImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/ResultSetLoaderImpl.java
index b875e7e..cc50729 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/ResultSetLoaderImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/ResultSetLoaderImpl.java
@@ -26,8 +26,8 @@ import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
 import org.apache.drill.exec.physical.rowSet.ResultVectorCache;
 import org.apache.drill.exec.physical.rowSet.RowSetLoader;
 import org.apache.drill.exec.physical.rowSet.impl.TupleState.RowState;
-import org.apache.drill.exec.record.TupleMetadata;
 import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.accessor.impl.HierarchicalFormatter;
 

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/RowSetLoaderImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/RowSetLoaderImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/RowSetLoaderImpl.java
index ec61ae7..dca749c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/RowSetLoaderImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/RowSetLoaderImpl.java
@@ -21,7 +21,7 @@ import java.util.ArrayList;
 
 import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
 import org.apache.drill.exec.physical.rowSet.RowSetLoader;
-import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.vector.accessor.writer.AbstractObjectWriter;
 import org.apache.drill.exec.vector.accessor.writer.AbstractTupleWriter;
 

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/SingleVectorState.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/SingleVectorState.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/SingleVectorState.java
index f6bc5f3..e813a70 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/SingleVectorState.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/SingleVectorState.java
@@ -18,7 +18,7 @@
 package org.apache.drill.exec.physical.rowSet.impl;
 
 import org.apache.drill.exec.expr.TypeHelper;
-import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.apache.drill.exec.vector.FixedWidthVector;
 import org.apache.drill.exec.vector.UInt4Vector;
 import org.apache.drill.exec.vector.ValueVector;

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/TupleState.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/TupleState.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/TupleState.java
index de41ee4..82f0437 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/TupleState.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/TupleState.java
@@ -23,11 +23,12 @@ import java.util.List;
 import org.apache.drill.exec.physical.rowSet.impl.ColumnState.BaseMapColumnState;
 import org.apache.drill.exec.physical.rowSet.impl.ColumnState.MapArrayColumnState;
 import org.apache.drill.exec.physical.rowSet.impl.ColumnState.MapColumnState;
-import org.apache.drill.exec.record.ColumnMetadata;
 import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.record.TupleMetadata;
-import org.apache.drill.exec.record.TupleSchema;
-import org.apache.drill.exec.record.TupleSchema.AbstractColumnMetadata;
+import org.apache.drill.exec.record.metadata.AbstractColumnMetadata;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.record.metadata.TupleSchema;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.accessor.ObjectType;
 import org.apache.drill.exec.vector.accessor.ObjectWriter;
@@ -182,7 +183,7 @@ public abstract class TupleState implements TupleWriterListener {
 
   @Override
   public ObjectWriter addColumn(TupleWriter tupleWriter, MaterializedField column) {
-    return addColumn(tupleWriter, TupleSchema.fromField(column));
+    return addColumn(tupleWriter, MetadataUtils.fromField(column));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/VectorContainerBuilder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/VectorContainerBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/VectorContainerBuilder.java
index faa68cb..7411176 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/VectorContainerBuilder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/VectorContainerBuilder.java
@@ -22,9 +22,9 @@ import java.util.List;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.physical.rowSet.impl.ColumnState.BaseMapColumnState;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
-import org.apache.drill.exec.record.ColumnMetadata;
-import org.apache.drill.exec.record.TupleMetadata;
-import org.apache.drill.exec.record.TupleSchema;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.record.metadata.TupleSchema;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.vector.UInt4Vector;
 import org.apache.drill.exec.vector.ValueVector;

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/BaseTupleModel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/BaseTupleModel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/BaseTupleModel.java
index 40da4ec..ff2481f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/BaseTupleModel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/BaseTupleModel.java
@@ -20,10 +20,10 @@ package org.apache.drill.exec.physical.rowSet.model;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.drill.exec.record.ColumnMetadata;
-import org.apache.drill.exec.record.TupleMetadata;
-import org.apache.drill.exec.record.TupleSchema;
-import org.apache.drill.exec.record.TupleSchema.AbstractColumnMetadata;
+import org.apache.drill.exec.record.metadata.AbstractColumnMetadata;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.record.metadata.TupleSchema;
 import org.apache.drill.exec.record.VectorContainer;
 
 /**

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/MetadataProvider.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/MetadataProvider.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/MetadataProvider.java
index bb5e18e..27e3c3d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/MetadataProvider.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/MetadataProvider.java
@@ -17,10 +17,10 @@
  */
 package org.apache.drill.exec.physical.rowSet.model;
 
-import org.apache.drill.exec.record.ColumnMetadata;
 import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.record.TupleMetadata;
-import org.apache.drill.exec.record.TupleSchema;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.record.metadata.TupleSchema;
 
 /**
  * Interface for retrieving and/or creating metadata given

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/SchemaInference.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/SchemaInference.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/SchemaInference.java
index 3db01dd..9096ec2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/SchemaInference.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/SchemaInference.java
@@ -21,11 +21,12 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.drill.common.types.TypeProtos.MinorType;
-import org.apache.drill.exec.record.ColumnMetadata;
 import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.record.TupleMetadata;
-import org.apache.drill.exec.record.TupleSchema;
 import org.apache.drill.exec.record.VectorContainer;
+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;
 
 /**
  * Produce a metadata schema from a vector container. Used when given a
@@ -40,14 +41,14 @@ public class SchemaInference {
       MaterializedField field = container.getValueVector(i).getField();
       columns.add(inferVector(field));
     }
-    return TupleSchema.fromColumns(columns);
+    return MetadataUtils.fromColumns(columns);
   }
 
   private ColumnMetadata inferVector(MaterializedField field) {
     if (field.getType().getMinorType() == MinorType.MAP) {
-      return TupleSchema.newMap(field, inferMapSchema(field));
+      return MetadataUtils.newMap(field, inferMapSchema(field));
     } else {
-      return TupleSchema.fromField(field);
+      return MetadataUtils.fromField(field);
     }
   }
 
@@ -56,6 +57,6 @@ public class SchemaInference {
     for (MaterializedField child : field.getChildren()) {
       columns.add(inferVector(child));
     }
-    return TupleSchema.fromColumns(columns);
+    return MetadataUtils.fromColumns(columns);
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/TupleModel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/TupleModel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/TupleModel.java
index 5fcba73..124119d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/TupleModel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/TupleModel.java
@@ -19,11 +19,11 @@ package org.apache.drill.exec.physical.rowSet.model;
 
 import javax.sql.RowSet;
 
-import org.apache.drill.exec.record.ColumnMetadata;
 import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.record.TupleMetadata;
-import org.apache.drill.exec.record.TupleSchema;
 import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.record.metadata.TupleSchema;
 import org.apache.drill.exec.vector.complex.AbstractMapVector;
 
 /**

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/single/BuildVectorsFromMetadata.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/single/BuildVectorsFromMetadata.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/single/BuildVectorsFromMetadata.java
index 30f60b3..50568db 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/single/BuildVectorsFromMetadata.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/single/BuildVectorsFromMetadata.java
@@ -20,9 +20,9 @@ package org.apache.drill.exec.physical.rowSet.model.single;
 import org.apache.drill.exec.expr.TypeHelper;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
-import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.record.TupleMetadata;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.complex.AbstractMapVector;

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/single/VectorAllocator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/single/VectorAllocator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/single/VectorAllocator.java
index e29a5cb..f4fc5d4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/single/VectorAllocator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/single/VectorAllocator.java
@@ -23,9 +23,9 @@ import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.physical.rowSet.model.MetadataProvider;
 import org.apache.drill.exec.physical.rowSet.model.MetadataProvider.MetadataCreator;
 import org.apache.drill.exec.physical.rowSet.model.MetadataProvider.MetadataRetrieval;
-import org.apache.drill.exec.record.ColumnMetadata;
-import org.apache.drill.exec.record.TupleMetadata;
 import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.vector.AllocationHelper;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.complex.AbstractMapVector;

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/main/java/org/apache/drill/exec/record/TupleSchema.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/TupleSchema.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/TupleSchema.java
deleted file mode 100644
index 27a88f0..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/TupleSchema.java
+++ /dev/null
@@ -1,534 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.record;
-
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.drill.common.types.TypeProtos.DataMode;
-import org.apache.drill.common.types.TypeProtos.MajorType;
-import org.apache.drill.common.types.TypeProtos.MinorType;
-import org.apache.drill.exec.expr.TypeHelper;
-import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
-
-/**
- * Defines the schema of a tuple: either the top-level row or a nested
- * "map" (really structure). A schema is a collection of columns (backed
- * by vectors in the loader itself.) Columns are accessible by name or
- * index. New columns may be added at any time; the new column takes the
- * next available index.
- */
-
-public class TupleSchema implements TupleMetadata {
-
-  /**
-   * Abstract definition of column metadata. Allows applications to create
-   * specialized forms of a column metadata object by extending from this
-   * abstract class.
-   * <p>
-   * Note that, by design, primitive columns do not have a link to their
-   * tuple parent, or their index within that parent. This allows the same
-   * metadata to be shared between two views of a tuple, perhaps physical
-   * and projected views. This restriction does not apply to map columns,
-   * since maps (and the row itself) will, by definition, differ between
-   * the two views.
-   */
-
-  public static abstract class AbstractColumnMetadata implements ColumnMetadata {
-
-    protected MaterializedField schema;
-    protected boolean projected = true;
-
-    /**
-     * Predicted number of elements per array entry. Default is
-     * taken from the often hard-coded value of 10.
-     */
-
-    protected int expectedElementCount = 1;
-
-    public AbstractColumnMetadata(MaterializedField schema) {
-      this.schema = schema;
-      if (isArray()) {
-        expectedElementCount = DEFAULT_ARRAY_SIZE;
-      }
-    }
-
-    public AbstractColumnMetadata(AbstractColumnMetadata from) {
-      schema = from.schema;
-      expectedElementCount = from.expectedElementCount;
-    }
-
-    protected void bind(TupleSchema parentTuple) { }
-
-    @Override
-    public MaterializedField schema() { return schema; }
-
-    public void replaceField(MaterializedField field) {
-      this.schema = field;
-    }
-    @Override
-    public String name() { return schema().getName(); }
-
-    @Override
-    public MajorType majorType() { return schema().getType(); }
-
-    @Override
-    public MinorType type() { return schema().getType().getMinorType(); }
-
-    @Override
-    public DataMode mode() { return schema().getDataMode(); }
-
-    @Override
-    public boolean isNullable() { return mode() == DataMode.OPTIONAL; }
-
-    @Override
-    public boolean isArray() { return mode() == DataMode.REPEATED; }
-
-    @Override
-    public boolean isList() { return false; }
-
-    @Override
-    public boolean isVariableWidth() {
-      MinorType type = type();
-      return type == MinorType.VARCHAR || type == MinorType.VAR16CHAR || type == MinorType.VARBINARY;
-    }
-
-    @Override
-    public boolean isEquivalent(ColumnMetadata other) {
-      return schema().isEquivalent(other.schema());
-    }
-
-    @Override
-    public int expectedWidth() { return 0; }
-
-    @Override
-    public void setExpectedWidth(int width) { }
-
-    @Override
-    public void setExpectedElementCount(int childCount) {
-      // The allocation utilities don't like an array size of zero, so set to
-      // 1 as the minimum. Adjusted to avoid trivial errors if the caller
-      // makes an error.
-
-      if (isArray()) {
-        expectedElementCount = Math.max(1, childCount);
-      }
-    }
-
-    @Override
-    public int expectedElementCount() { return expectedElementCount; }
-
-    @Override
-    public void setProjected(boolean projected) {
-      this.projected = projected;
-    }
-
-    @Override
-    public boolean isProjected() { return projected; }
-
-    @Override
-    public String toString() {
-      StringBuilder buf = new StringBuilder()
-          .append("[")
-          .append(getClass().getSimpleName())
-          .append(" ")
-          .append(schema().toString())
-          .append(",")
-          .append(projected ? "" : "not ")
-          .append("projected");
-      if (isArray()) {
-        buf.append(", cardinality: ")
-           .append(expectedElementCount);
-      }
-      return buf
-          .append("]")
-          .toString();
-    }
-
-    public abstract AbstractColumnMetadata copy();
-  }
-
-  /**
-   * Primitive (non-map) column. Describes non-nullable, nullable and
-   * array types (which differ only in mode, but not in metadata structure.)
-   */
-
-  public static class PrimitiveColumnMetadata extends AbstractColumnMetadata {
-
-    protected int expectedWidth;
-
-    public PrimitiveColumnMetadata(MaterializedField schema) {
-      super(schema);
-      expectedWidth = TypeHelper.getSize(majorType());
-      if (isVariableWidth()) {
-
-        // The above getSize() method uses the deprecated getWidth()
-        // method to get the expected VarChar size. If zero (which
-        // it will be), try the revised precision field.
-
-        int precision = majorType().getPrecision();
-        if (precision > 0) {
-          expectedWidth = precision;
-        } else {
-          // TypeHelper includes the offset vector width
-
-          expectedWidth = expectedWidth - 4;
-        }
-      }
-    }
-
-    public PrimitiveColumnMetadata(PrimitiveColumnMetadata from) {
-      super(from);
-      expectedWidth = from.expectedWidth;
-    }
-
-    @Override
-    public AbstractColumnMetadata copy() {
-      return new PrimitiveColumnMetadata(this);
-    }
-
-    @Override
-    public ColumnMetadata.StructureType structureType() { return ColumnMetadata.StructureType.PRIMITIVE; }
-
-    @Override
-    public TupleMetadata mapSchema() { return null; }
-
-    @Override
-    public boolean isMap() { return false; }
-
-    @Override
-    public int expectedWidth() { return expectedWidth; }
-
-    @Override
-    public void setExpectedWidth(int width) {
-      // The allocation utilities don't like a width of zero, so set to
-      // 1 as the minimum. Adjusted to avoid trivial errors if the caller
-      // makes an error.
-
-      if (isVariableWidth()) {
-        expectedWidth = Math.max(1, width);
-      }
-    }
-
-    @Override
-    public ColumnMetadata cloneEmpty() {
-      return new PrimitiveColumnMetadata(this);
-    }
-  }
-
-  /**
-   * Describes a map and repeated map. Both are tuples that have a tuple
-   * schema as part of the column definition.
-   */
-
-  public static class MapColumnMetadata extends AbstractColumnMetadata {
-    private TupleMetadata parentTuple;
-    private final TupleSchema mapSchema;
-
-    /**
-     * Build a new map column from the field provided
-     *
-     * @param schema materialized field description of the map
-     */
-
-    public MapColumnMetadata(MaterializedField schema) {
-      this(schema, null);
-    }
-
-    /**
-     * Build a map column metadata by cloning the type information (but not
-     * the children) of the materialized field provided. Use the hints
-     * provided.
-     *
-     * @param schema the schema to use
-     * @param hints metadata hints for this column
-     */
-
-    private MapColumnMetadata(MaterializedField schema, TupleSchema mapSchema) {
-      super(schema);
-      if (mapSchema == null) {
-        this.mapSchema = new TupleSchema();
-      } else {
-        this.mapSchema = mapSchema;
-      }
-      this.mapSchema.bind(this);
-    }
-
-    @Override
-    public AbstractColumnMetadata copy() {
-      return new MapColumnMetadata(schema, (TupleSchema) mapSchema.copy());
-    }
-
-    @Override
-    protected void bind(TupleSchema parentTuple) {
-      this.parentTuple = parentTuple;
-    }
-
-    @Override
-    public ColumnMetadata.StructureType structureType() { return ColumnMetadata.StructureType.TUPLE; }
-
-    @Override
-    public TupleMetadata mapSchema() { return mapSchema; }
-
-    @Override
-    public int expectedWidth() { return 0; }
-
-    @Override
-    public boolean isMap() { return true; }
-
-    public TupleMetadata parentTuple() { return parentTuple; }
-
-    public TupleSchema mapSchemaImpl() { return mapSchema; }
-
-    @Override
-    public ColumnMetadata cloneEmpty() {
-      return new MapColumnMetadata(schema().cloneEmpty(), null);
-    }
-  }
-
-  private MapColumnMetadata parentMap;
-  private final TupleNameSpace<ColumnMetadata> nameSpace = new TupleNameSpace<>();
-
-  public void bind(MapColumnMetadata parentMap) {
-    this.parentMap = parentMap;
-  }
-
-  public static TupleSchema fromFields(Iterable<MaterializedField> fields) {
-    TupleSchema tuple = new TupleSchema();
-    for (MaterializedField field : fields) {
-      tuple.add(field);
-    }
-    return tuple;
-  }
-
-  public TupleMetadata copy() {
-    TupleMetadata tuple = new TupleSchema();
-    for (ColumnMetadata md : this) {
-      tuple.addColumn(((AbstractColumnMetadata) md).copy());
-    }
-    return tuple;
-  }
-
-  /**
-   * Create a column metadata object that holds the given
-   * {@link MaterializedField}. The type of the object will be either a
-   * primitive or map column, depending on the field's type.
-   *
-   * @param field the materialized field to wrap
-   * @return the column metadata that wraps the field
-   */
-
-  public static AbstractColumnMetadata fromField(MaterializedField field) {
-    if (field.getType().getMinorType() == MinorType.MAP) {
-      return newMap(field);
-    } else {
-      return new PrimitiveColumnMetadata(field);
-    }
-  }
-
-  public static AbstractColumnMetadata fromView(MaterializedField field) {
-    if (field.getType().getMinorType() == MinorType.MAP) {
-      return new MapColumnMetadata(field, null);
-    } else {
-      return new PrimitiveColumnMetadata(field);
-    }
-  }
-
-  /**
-   * Create a tuple given the list of columns that make up the tuple.
-   * Creates nested maps as needed.
-   *
-   * @param columns list of columns that make up the tuple
-   * @return a tuple metadata object that contains the columns
-   */
-
-  public static TupleSchema fromColumns(List<ColumnMetadata> columns) {
-    TupleSchema tuple = new TupleSchema();
-    for (ColumnMetadata column : columns) {
-      tuple.add((AbstractColumnMetadata) column);
-    }
-    return tuple;
-  }
-
-  /**
-   * Create a column metadata object for a map column, given the
-   * {@link MaterializedField} that describes the column, and a list
-   * of column metadata objects that describe the columns in the map.
-   *
-   * @param field the materialized field that describes the map column
-   * @param schema metadata that describes the tuple of columns in
-   * the map
-   * @return a map column metadata for the map
-   */
-
-  public static MapColumnMetadata newMap(MaterializedField field, TupleSchema schema) {
-    return new MapColumnMetadata(field, schema);
-  }
-
-  public static MapColumnMetadata newMap(MaterializedField field) {
-    return new MapColumnMetadata(field, fromFields(field.getChildren()));
-  }
-
-  @Override
-  public ColumnMetadata add(MaterializedField field) {
-    AbstractColumnMetadata md = fromField(field);
-    add(md);
-    return md;
-  }
-
-  public ColumnMetadata addView(MaterializedField field) {
-    AbstractColumnMetadata md = fromView(field);
-    add(md);
-    return md;
-  }
-
-  /**
-   * Add a column metadata column created by the caller. Used for specialized
-   * cases beyond those handled by {@link #add(MaterializedField)}.
-   *
-   * @param md the custom column metadata which must have the correct
-   * index set (from {@link #size()}
-   */
-
-  public void add(AbstractColumnMetadata md) {
-    md.bind(this);
-    nameSpace.add(md.name(), md);
-    if (parentMap != null) {
-      parentMap.schema.addChild(md.schema());
-    }
-  }
-
-  @Override
-  public int addColumn(ColumnMetadata column) {
-    add((AbstractColumnMetadata) column);
-    return size() - 1;
-  }
-
-  @Override
-  public MaterializedField column(String name) {
-    ColumnMetadata md = metadata(name);
-    return md == null ? null : md.schema();
-  }
-
-  @Override
-  public ColumnMetadata metadata(String name) {
-    return nameSpace.get(name);
-  }
-
-  @Override
-  public int index(String name) {
-    return nameSpace.indexOf(name);
-  }
-
-  @Override
-  public MaterializedField column(int index) {
-    return metadata(index).schema();
-  }
-
-  @Override
-  public ColumnMetadata metadata(int index) {
-    return nameSpace.get(index);
-  }
-
-  @Override
-  public MapColumnMetadata parent() { return parentMap; }
-
-  @Override
-  public int size() { return nameSpace.count(); }
-
-  @Override
-  public boolean isEmpty() { return nameSpace.count( ) == 0; }
-
-  @Override
-  public Iterator<ColumnMetadata> iterator() {
-    return nameSpace.iterator();
-  }
-
-  @Override
-  public boolean isEquivalent(TupleMetadata other) {
-    TupleSchema otherSchema = (TupleSchema) other;
-    if (nameSpace.count() != otherSchema.nameSpace.count()) {
-      return false;
-    }
-    for (int i = 0; i < nameSpace.count(); i++) {
-      if (! nameSpace.get(i).isEquivalent(otherSchema.nameSpace.get(i))) {
-        return false;
-      }
-    }
-    return true;
-  }
-
-  @Override
-  public List<MaterializedField> toFieldList() {
-    List<MaterializedField> cols = new ArrayList<>();
-    for (ColumnMetadata md : nameSpace) {
-      cols.add(md.schema());
-    }
-    return cols;
-  }
-
-  public BatchSchema toBatchSchema(SelectionVectorMode svMode) {
-    return new BatchSchema(svMode, toFieldList());
-  }
-
-  @Override
-  public String fullName(int index) {
-    return fullName(metadata(index));
-  }
-
-  @Override
-  public String fullName(ColumnMetadata column) {
-    String quotedName = column.name();
-    if (quotedName.contains(".")) {
-      quotedName = "`" + quotedName + "`";
-    }
-    if (isRoot()) {
-      return column.name();
-    } else {
-      return fullName() + "." + quotedName;
-    }
-  }
-
-  public String fullName() {
-    if (isRoot()) {
-      return "<root>";
-    } else {
-      return parentMap.parentTuple().fullName(parentMap);
-    }
-  }
-
-  public boolean isRoot() { return parentMap == null; }
-
-  @Override
-  public String toString() {
-    StringBuilder buf = new StringBuilder()
-        .append("[")
-        .append(getClass().getSimpleName())
-        .append(" ");
-    boolean first = true;
-    for (ColumnMetadata md : nameSpace) {
-      if (! first) {
-        buf.append(", ");
-      }
-      buf.append(md.toString());
-    }
-    buf.append("]");
-    return buf.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/AbstractColumnMetadata.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/AbstractColumnMetadata.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/AbstractColumnMetadata.java
new file mode 100644
index 0000000..2073c80
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/AbstractColumnMetadata.java
@@ -0,0 +1,206 @@
+/*
+ * 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.record.metadata;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.record.MaterializedField;
+
+/**
+ * Abstract definition of column metadata. Allows applications to create
+ * specialized forms of a column metadata object by extending from this
+ * abstract class.
+ * <p>
+ * Note that, by design, primitive columns do not have a link to their
+ * tuple parent, or their index within that parent. This allows the same
+ * metadata to be shared between two views of a tuple, perhaps physical
+ * and projected views. This restriction does not apply to map columns,
+ * since maps (and the row itself) will, by definition, differ between
+ * the two views.
+ */
+
+public abstract class AbstractColumnMetadata implements ColumnMetadata {
+
+  // Capture the key schema information. We cannot use the MaterializedField
+  // or MajorType because then encode child information that we encode here
+  // as a child schema. Keeping the two in sync is nearly impossible.
+
+  protected final String name;
+  protected final MinorType type;
+  protected final DataMode mode;
+  protected final int precision;
+  protected final int scale;
+  protected boolean projected = true;
+
+  /**
+   * Predicted number of elements per array entry. Default is
+   * taken from the often hard-coded value of 10.
+   */
+
+  protected int expectedElementCount = 1;
+
+  public AbstractColumnMetadata(MaterializedField schema) {
+    name = schema.getName();
+    MajorType majorType = schema.getType();
+    type = majorType.getMinorType();
+    mode = majorType.getMode();
+    precision = majorType.getPrecision();
+    scale = majorType.getScale();
+    if (isArray()) {
+      expectedElementCount = DEFAULT_ARRAY_SIZE;
+    }
+  }
+
+  public AbstractColumnMetadata(String name, MinorType type, DataMode mode) {
+    this.name = name;
+    this.type = type;
+    this.mode = mode;
+    precision = 0;
+    scale = 0;
+    if (isArray()) {
+      expectedElementCount = DEFAULT_ARRAY_SIZE;
+    }
+  }
+
+  public AbstractColumnMetadata(AbstractColumnMetadata from) {
+    name = from.name;
+    type = from.type;
+    mode = from.mode;
+    precision = from.precision;
+    scale = from.scale;
+    expectedElementCount = from.expectedElementCount;
+  }
+
+  protected void bind(TupleSchema parentTuple) { }
+
+  @Override
+  public String name() { return name; }
+
+  @Override
+  public MinorType type() { return type; }
+
+  @Override
+  public MajorType majorType() {
+    return MajorType.newBuilder()
+        .setMinorType(type())
+        .setMode(mode())
+        .build();
+  }
+
+  @Override
+  public DataMode mode() { return mode; }
+
+  @Override
+  public boolean isNullable() { return mode() == DataMode.OPTIONAL; }
+
+  @Override
+  public boolean isArray() { return mode() == DataMode.REPEATED; }
+
+  @Override
+  public int dimensions() { return isArray() ? 1 : 0; }
+
+  @Override
+  public boolean isMap() { return false; }
+
+  @Override
+  public boolean isVariant() { return false; }
+
+  @Override
+  public TupleMetadata mapSchema() { return null; }
+
+  @Override
+  public VariantMetadata variantSchema() { return null; }
+
+  @Override
+  public ColumnMetadata childSchema() { return null; }
+
+  @Override
+  public boolean isVariableWidth() {
+    MinorType type = type();
+    return type == MinorType.VARCHAR || type == MinorType.VAR16CHAR || type == MinorType.VARBINARY;
+  }
+
+  @Override
+  public boolean isEquivalent(ColumnMetadata other) {
+    return schema().isEquivalent(other.schema());
+  }
+
+  @Override
+  public int expectedWidth() { return 0; }
+
+  @Override
+  public void setExpectedWidth(int width) { }
+
+  @Override
+  public int precision() { return 0; }
+
+  @Override
+  public int scale() { return 0; }
+
+  @Override
+  public void setExpectedElementCount(int childCount) {
+    // The allocation utilities don't like an array size of zero, so set to
+    // 1 as the minimum. Adjusted to avoid trivial errors if the caller
+    // makes an error.
+
+    if (isArray()) {
+      expectedElementCount = Math.max(1, childCount);
+    }
+  }
+
+  @Override
+  public int expectedElementCount() { return expectedElementCount; }
+
+  @Override
+  public void setProjected(boolean projected) {
+    this.projected = projected;
+  }
+
+  @Override
+  public boolean isProjected() { return projected; }
+
+  @Override
+  public String toString() {
+    StringBuilder buf = new StringBuilder()
+        .append("[")
+        .append(getClass().getSimpleName())
+        .append(" ")
+        .append(schema().toString())
+        .append(", ")
+        .append(projected ? "" : "not ")
+        .append("projected");
+    if (isArray()) {
+      buf.append(", cardinality: ")
+         .append(expectedElementCount);
+    }
+    if (variantSchema() != null) {
+      buf.append(", variant: ")
+         .append(variantSchema().toString());
+    }
+    if (mapSchema() != null) {
+      buf.append(", schema: ")
+         .append(mapSchema().toString());
+    }
+    return buf
+        .append("]")
+        .toString();
+  }
+
+  public abstract AbstractColumnMetadata copy();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/MapColumnMetadata.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/MapColumnMetadata.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/MapColumnMetadata.java
new file mode 100644
index 0000000..795fd7f
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/MapColumnMetadata.java
@@ -0,0 +1,126 @@
+/*
+ * 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.record.metadata;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.record.MaterializedField;
+
+/**
+ * Describes a map and repeated map. Both are tuples that have a tuple
+ * schema as part of the column definition.
+ */
+
+public class MapColumnMetadata extends AbstractColumnMetadata {
+  private TupleMetadata parentTuple;
+  private final TupleSchema mapSchema;
+
+  /**
+   * Build a new map column from the field provided
+   *
+   * @param schema materialized field description of the map
+   */
+
+  public MapColumnMetadata(MaterializedField schema) {
+    this(schema, null);
+  }
+
+  /**
+   * Build a map column metadata by cloning the type information (but not
+   * the children) of the materialized field provided. Use the hints
+   * provided.
+   *
+   * @param schema the schema to use
+   * @param hints metadata hints for this column
+   */
+
+  MapColumnMetadata(MaterializedField schema, TupleSchema mapSchema) {
+    super(schema);
+    if (mapSchema == null) {
+      this.mapSchema = new TupleSchema();
+    } else {
+      this.mapSchema = mapSchema;
+    }
+    this.mapSchema.bind(this);
+  }
+
+  public MapColumnMetadata(MapColumnMetadata from) {
+    super(from);
+    mapSchema = (TupleSchema) from.mapSchema.copy();
+  }
+
+  public MapColumnMetadata(String name, DataMode mode,
+      TupleSchema mapSchema) {
+    super(name, MinorType.MAP, mode);
+    if (mapSchema == null) {
+      this.mapSchema = new TupleSchema();
+    } else {
+      this.mapSchema = mapSchema;
+    }
+  }
+
+  @Override
+  public AbstractColumnMetadata copy() {
+    return new MapColumnMetadata(this);
+  }
+
+  @Override
+  protected void bind(TupleSchema parentTuple) {
+    this.parentTuple = parentTuple;
+  }
+
+  @Override
+  public ColumnMetadata.StructureType structureType() { return ColumnMetadata.StructureType.TUPLE; }
+
+  @Override
+  public TupleMetadata mapSchema() { return mapSchema; }
+
+  @Override
+  public int expectedWidth() { return 0; }
+
+  @Override
+  public boolean isMap() { return true; }
+
+  public TupleMetadata parentTuple() { return parentTuple; }
+
+  public TupleSchema mapSchemaImpl() { return mapSchema; }
+
+  @Override
+  public ColumnMetadata cloneEmpty() {
+    return new MapColumnMetadata(name, mode, new TupleSchema());
+  }
+
+  @Override
+  public MaterializedField schema() {
+    MaterializedField field = emptySchema();
+    for (MaterializedField member : mapSchema.toFieldList()) {
+      field.addChild(member);
+    }
+    return field;
+  }
+
+  @Override
+  public MaterializedField emptySchema() {
+   return MaterializedField.create(name,
+        MajorType.newBuilder()
+          .setMinorType(type)
+          .setMode(mode)
+          .build());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/MetadataUtils.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/MetadataUtils.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/MetadataUtils.java
new file mode 100644
index 0000000..8c8dea7
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/MetadataUtils.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.record.metadata;
+
+import java.util.List;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.MaterializedField;
+
+public class MetadataUtils {
+
+  public static TupleSchema fromFields(Iterable<MaterializedField> fields) {
+    TupleSchema tuple = new TupleSchema();
+    for (MaterializedField field : fields) {
+      tuple.add(field);
+    }
+    return tuple;
+  }
+
+  /**
+   * Create a column metadata object that holds the given
+   * {@link MaterializedField}. The type of the object will be either a
+   * primitive or map column, depending on the field's type. The logic
+   * here mimics the code as written, which is very messy in some places.
+   *
+   * @param field the materialized field to wrap
+   * @return the column metadata that wraps the field
+   */
+
+  public static AbstractColumnMetadata fromField(MaterializedField field) {
+    MinorType type = field.getType().getMinorType();
+    switch (type) {
+    case MAP:
+      return MetadataUtils.newMap(field);
+    case UNION:
+      if (field.getType().getMode() != DataMode.OPTIONAL) {
+        throw new UnsupportedOperationException(type.name() + " type must be nullable");
+      }
+      return new VariantColumnMetadata(field);
+    case LIST:
+      switch (field.getType().getMode()) {
+      case OPTIONAL:
+        return new VariantColumnMetadata(field);
+      case REPEATED:
+
+        // Not a list at all, but rather the second (or third...)
+        // dimension on a repeated type.
+
+        return new RepeatedListColumnMetadata(field);
+      default:
+
+        // List of unions (or a degenerate union of a single type.)
+        // Not supported in Drill.
+
+        throw new UnsupportedOperationException(
+            String.format("Unsupported mode %s for type %s",
+                field.getType().getMode().name(),
+                type.name()));
+      }
+    default:
+      return new PrimitiveColumnMetadata(field);
+    }
+  }
+
+  public static AbstractColumnMetadata fromView(MaterializedField field) {
+    if (field.getType().getMinorType() == MinorType.MAP) {
+      return new MapColumnMetadata(field, null);
+    } else {
+      return new PrimitiveColumnMetadata(field);
+    }
+  }
+
+  /**
+   * Create a tuple given the list of columns that make up the tuple.
+   * Creates nested maps as needed.
+   *
+   * @param columns list of columns that make up the tuple
+   * @return a tuple metadata object that contains the columns
+   */
+
+  public static TupleSchema fromColumns(List<ColumnMetadata> columns) {
+    TupleSchema tuple = new TupleSchema();
+    for (ColumnMetadata column : columns) {
+      tuple.add((AbstractColumnMetadata) column);
+    }
+    return tuple;
+  }
+
+  public static TupleMetadata fromBatchSchema(BatchSchema batchSchema) {
+    TupleSchema tuple = new TupleSchema();
+    for (MaterializedField field : batchSchema) {
+      tuple.add(fromView(field));
+    }
+    return tuple;
+  }
+
+  /**
+   * Create a column metadata object for a map column, given the
+   * {@link MaterializedField} that describes the column, and a list
+   * of column metadata objects that describe the columns in the map.
+   *
+   * @param field the materialized field that describes the map column
+   * @param schema metadata that describes the tuple of columns in
+   * the map
+   * @return a map column metadata for the map
+   */
+
+  public static MapColumnMetadata newMap(MaterializedField field, TupleSchema schema) {
+    return new MapColumnMetadata(field, schema);
+  }
+
+  public static MapColumnMetadata newMap(MaterializedField field) {
+    return new MapColumnMetadata(field, fromFields(field.getChildren()));
+  }
+
+  public static MapColumnMetadata newMap(String name, TupleMetadata schema) {
+    return new MapColumnMetadata(name, DataMode.REQUIRED, (TupleSchema) schema);
+  }
+
+  public static VariantColumnMetadata newVariant(MaterializedField field, VariantSchema schema) {
+    return new VariantColumnMetadata(field, schema);
+  }
+
+  public static VariantColumnMetadata newVariant(String name, DataMode cardinality) {
+    switch (cardinality) {
+    case OPTIONAL:
+      return new VariantColumnMetadata(name, MinorType.UNION, new VariantSchema());
+    case REPEATED:
+      return new VariantColumnMetadata(name, MinorType.LIST, new VariantSchema());
+    default:
+      throw new IllegalArgumentException();
+    }
+  }
+
+  public static RepeatedListColumnMetadata newRepeatedList(String name, AbstractColumnMetadata child) {
+    return new RepeatedListColumnMetadata(name, child);
+  }
+
+  public static AbstractColumnMetadata newMapArray(String name, TupleMetadata schema) {
+    return new MapColumnMetadata(name, DataMode.REPEATED, (TupleSchema) schema);
+  }
+
+  public static PrimitiveColumnMetadata newScalar(String name, MinorType type,
+      DataMode mode) {
+    assert type != MinorType.MAP && type != MinorType.UNION && type != MinorType.LIST;
+    return new PrimitiveColumnMetadata(name, type, mode);
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/PrimitiveColumnMetadata.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/PrimitiveColumnMetadata.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/PrimitiveColumnMetadata.java
new file mode 100644
index 0000000..f0c25eb
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/PrimitiveColumnMetadata.java
@@ -0,0 +1,131 @@
+/*
+ * 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.record.metadata;
+
+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.expr.TypeHelper;
+import org.apache.drill.exec.record.MaterializedField;
+
+/**
+ * Primitive (non-map) column. Describes non-nullable, nullable and
+ * array types (which differ only in mode, but not in metadata structure.)
+ */
+
+public class PrimitiveColumnMetadata extends AbstractColumnMetadata {
+
+  protected int expectedWidth;
+
+  public PrimitiveColumnMetadata(MaterializedField schema) {
+    super(schema);
+    expectedWidth = estimateWidth(schema.getType());
+  }
+
+  public PrimitiveColumnMetadata(String name, MinorType type, DataMode mode) {
+    super(name, type, mode);
+    expectedWidth = estimateWidth(Types.withMode(type, mode));
+  }
+
+  private int estimateWidth(MajorType majorType) {
+    if (type() == MinorType.NULL || type() == MinorType.LATE) {
+      return 0;
+    } else if (isVariableWidth()) {
+
+      // The above getSize() method uses the deprecated getWidth()
+      // method to get the expected VarChar size. If zero (which
+      // it will be), try the revised precision field.
+
+      int precision = majorType.getPrecision();
+      if (precision > 0) {
+        return precision;
+      } else {
+        // TypeHelper includes the offset vector width
+
+        return TypeHelper.getSize(majorType) - 4;
+      }
+    } else {
+      return TypeHelper.getSize(majorType);
+    }
+  }
+
+  public PrimitiveColumnMetadata(PrimitiveColumnMetadata from) {
+    super(from);
+    expectedWidth = from.expectedWidth;
+  }
+
+  @Override
+  public AbstractColumnMetadata copy() {
+    return new PrimitiveColumnMetadata(this);
+  }
+
+  @Override
+  public ColumnMetadata.StructureType structureType() { return ColumnMetadata.StructureType.PRIMITIVE; }
+
+  @Override
+  public int expectedWidth() { return expectedWidth; }
+
+  @Override
+  public int precision() { return precision; }
+
+  @Override
+  public int scale() { return scale; }
+
+  @Override
+  public void setExpectedWidth(int width) {
+    // The allocation utilities don't like a width of zero, so set to
+    // 1 as the minimum. Adjusted to avoid trivial errors if the caller
+    // makes an error.
+
+    if (isVariableWidth()) {
+      expectedWidth = Math.max(1, width);
+    }
+  }
+
+  @Override
+  public ColumnMetadata cloneEmpty() {
+    return new PrimitiveColumnMetadata(this);
+  }
+
+  public ColumnMetadata mergeWith(MaterializedField field) {
+    PrimitiveColumnMetadata merged = new PrimitiveColumnMetadata(field);
+    merged.setExpectedElementCount(expectedElementCount);
+    merged.setExpectedWidth(Math.max(expectedWidth, field.getPrecision()));
+    merged.setProjected(projected);
+    return merged;
+  }
+
+  @Override
+  public MajorType majorType() {
+    return MajorType.newBuilder()
+        .setMinorType(type)
+        .setMode(mode)
+        .setPrecision(precision)
+        .setScale(scale)
+        .build();
+  }
+
+  @Override
+  public MaterializedField schema() {
+    return MaterializedField.create(name, majorType());
+  }
+
+  @Override
+  public MaterializedField emptySchema() { return schema(); }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/RepeatedListColumnMetadata.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/RepeatedListColumnMetadata.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/RepeatedListColumnMetadata.java
new file mode 100644
index 0000000..a126965
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/RepeatedListColumnMetadata.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.record.metadata;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.record.MaterializedField;
+
+import com.google.common.base.Preconditions;
+
+public class RepeatedListColumnMetadata extends AbstractColumnMetadata {
+
+  /**
+   * Indicates we don't know the number of dimensions.
+   */
+
+  public static final int UNKNOWN_DIMENSIONS = -1;
+
+  private AbstractColumnMetadata childSchema;
+
+  public RepeatedListColumnMetadata(MaterializedField field) {
+    super(field);
+    Preconditions.checkArgument(field.getType().getMinorType() == MinorType.LIST);
+    Preconditions.checkArgument(field.getType().getMode() == DataMode.REPEATED);
+    Preconditions.checkArgument(field.getChildren().size() <= 1);
+    if (! field.getChildren().isEmpty()) {
+      childSchema = MetadataUtils.fromField(field.getChildren().iterator().next());
+      Preconditions.checkArgument(childSchema.isArray());
+    }
+  }
+
+  public RepeatedListColumnMetadata(String name, AbstractColumnMetadata childSchema) {
+    super(name, MinorType.LIST, DataMode.REPEATED);
+    if (childSchema != null) {
+      Preconditions.checkArgument(childSchema.isArray());
+    }
+    this.childSchema = childSchema;
+  }
+
+  public void childSchema(ColumnMetadata childMetadata) {
+    Preconditions.checkState(childSchema == null);
+    Preconditions.checkArgument(childMetadata.mode() == DataMode.REPEATED);
+    childSchema = (AbstractColumnMetadata) childMetadata;
+  }
+
+  @Override
+  public StructureType structureType() { return StructureType.MULTI_ARRAY; }
+
+  @Override
+  public MaterializedField schema() {
+    MaterializedField field = emptySchema();
+    if (childSchema != null) {
+      field.addChild(childSchema.schema());
+    }
+    return field;
+  }
+
+  @Override
+  public MaterializedField emptySchema() {
+    return MaterializedField.create(name(), majorType());
+  }
+
+  @Override
+  public ColumnMetadata cloneEmpty() {
+    return new RepeatedListColumnMetadata(name, null);
+  }
+
+  @Override
+  public AbstractColumnMetadata copy() {
+    return new RepeatedListColumnMetadata(name, childSchema);
+  }
+
+  @Override
+  public ColumnMetadata childSchema() { return childSchema; }
+
+  @Override
+  public int dimensions() {
+
+    // If there is no child, then we don't know the
+    // dimensionality.
+
+    return childSchema == null ? UNKNOWN_DIMENSIONS
+        : childSchema.dimensions() + 1;
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/TupleSchema.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/TupleSchema.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/TupleSchema.java
new file mode 100644
index 0000000..0c69dbc
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/TupleSchema.java
@@ -0,0 +1,196 @@
+/*
+ * 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.record.metadata;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+
+/**
+ * Defines the schema of a tuple: either the top-level row or a nested
+ * "map" (really structure). A schema is a collection of columns (backed
+ * by vectors in the loader itself.) Columns are accessible by name or
+ * index. New columns may be added at any time; the new column takes the
+ * next available index.
+ */
+
+public class TupleSchema implements TupleMetadata {
+
+  private MapColumnMetadata parentMap;
+  private final TupleNameSpace<ColumnMetadata> nameSpace = new TupleNameSpace<>();
+
+  public void bind(MapColumnMetadata parentMap) {
+    this.parentMap = parentMap;
+  }
+
+  public TupleMetadata copy() {
+    TupleMetadata tuple = new TupleSchema();
+    for (ColumnMetadata md : this) {
+      tuple.addColumn(((AbstractColumnMetadata) md).copy());
+    }
+    return tuple;
+  }
+
+  @Override
+  public ColumnMetadata add(MaterializedField field) {
+    AbstractColumnMetadata md = MetadataUtils.fromField(field);
+    add(md);
+    return md;
+  }
+
+  public ColumnMetadata addView(MaterializedField field) {
+    AbstractColumnMetadata md = MetadataUtils.fromView(field);
+    add(md);
+    return md;
+  }
+
+  /**
+   * Add a column metadata column created by the caller. Used for specialized
+   * cases beyond those handled by {@link #add(MaterializedField)}.
+   *
+   * @param md the custom column metadata which must have the correct
+   * index set (from {@link #size()}
+   */
+
+  public void add(AbstractColumnMetadata md) {
+    md.bind(this);
+    nameSpace.add(md.name(), md);
+  }
+
+  @Override
+  public int addColumn(ColumnMetadata column) {
+    add((AbstractColumnMetadata) column);
+    return size() - 1;
+  }
+
+  @Override
+  public MaterializedField column(String name) {
+    ColumnMetadata md = metadata(name);
+    return md == null ? null : md.schema();
+  }
+
+  @Override
+  public ColumnMetadata metadata(String name) {
+    return nameSpace.get(name);
+  }
+
+  @Override
+  public int index(String name) {
+    return nameSpace.indexOf(name);
+  }
+
+  @Override
+  public MaterializedField column(int index) {
+    return metadata(index).schema();
+  }
+
+  @Override
+  public ColumnMetadata metadata(int index) {
+    return nameSpace.get(index);
+  }
+
+  @Override
+  public MapColumnMetadata parent() { return parentMap; }
+
+  @Override
+  public int size() { return nameSpace.count(); }
+
+  @Override
+  public boolean isEmpty() { return nameSpace.count( ) == 0; }
+
+  @Override
+  public Iterator<ColumnMetadata> iterator() {
+    return nameSpace.iterator();
+  }
+
+  @Override
+  public boolean isEquivalent(TupleMetadata other) {
+    TupleSchema otherSchema = (TupleSchema) other;
+    if (nameSpace.count() != otherSchema.nameSpace.count()) {
+      return false;
+    }
+    for (int i = 0; i < nameSpace.count(); i++) {
+      if (! nameSpace.get(i).isEquivalent(otherSchema.nameSpace.get(i))) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  @Override
+  public List<MaterializedField> toFieldList() {
+    List<MaterializedField> cols = new ArrayList<>();
+    for (ColumnMetadata md : nameSpace) {
+      cols.add(md.schema());
+    }
+    return cols;
+  }
+
+  public BatchSchema toBatchSchema(SelectionVectorMode svMode) {
+    return new BatchSchema(svMode, toFieldList());
+  }
+
+  @Override
+  public String fullName(int index) {
+    return fullName(metadata(index));
+  }
+
+  @Override
+  public String fullName(ColumnMetadata column) {
+    String quotedName = column.name();
+    if (quotedName.contains(".")) {
+      quotedName = "`" + quotedName + "`";
+    }
+    if (isRoot()) {
+      return column.name();
+    } else {
+      return fullName() + "." + quotedName;
+    }
+  }
+
+  public String fullName() {
+    if (isRoot()) {
+      return "<root>";
+    } else {
+      return parentMap.parentTuple().fullName(parentMap);
+    }
+  }
+
+  public boolean isRoot() { return parentMap == null; }
+
+  @Override
+  public String toString() {
+    StringBuilder buf = new StringBuilder()
+        .append("[")
+        .append(getClass().getSimpleName())
+        .append(" ");
+    boolean first = true;
+    for (ColumnMetadata md : nameSpace) {
+      if (! first) {
+        buf.append(", ");
+      }
+      buf.append(md.toString());
+    }
+    buf.append("]");
+    return buf.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/VariantColumnMetadata.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/VariantColumnMetadata.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/VariantColumnMetadata.java
new file mode 100644
index 0000000..b417107
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/VariantColumnMetadata.java
@@ -0,0 +1,136 @@
+/*
+ * 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.record.metadata;
+
+import java.util.List;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.record.MaterializedField;
+
+public class VariantColumnMetadata extends AbstractColumnMetadata {
+
+  private final VariantSchema variantSchema;
+
+  public VariantColumnMetadata(MaterializedField schema) {
+    super(schema);
+    variantSchema = new VariantSchema();
+    variantSchema.bind(this);
+    List<MinorType> types = null;
+    if (type() == MinorType.UNION) {
+      types = schema.getType().getSubTypeList();
+    } else {
+      assert type() == MinorType.LIST;
+      MaterializedField child;
+      MinorType childType;
+      if (schema.getChildren().isEmpty()) {
+        child = null;
+        childType = MinorType.LATE;
+      } else {
+        child = schema.getChildren().iterator().next();
+        childType = child.getType().getMinorType();
+      }
+      switch (childType) {
+      case UNION:
+
+        // List contains a union.
+
+        types = child.getType().getSubTypeList();
+        break;
+
+      case LATE:
+
+        // List has no type.
+
+        return;
+
+      default:
+
+        // List contains a single non-null type.
+
+        variantSchema.addType(MetadataUtils.fromField(child));
+        return;
+      }
+    }
+    if (types == null) {
+      return;
+    }
+    for (MinorType type : types) {
+      variantSchema.addType(type);
+    }
+  }
+
+  public VariantColumnMetadata(MaterializedField schema, VariantSchema variantSchema) {
+    super(schema);
+    this.variantSchema = variantSchema;
+  }
+
+  public VariantColumnMetadata(String name, MinorType type, VariantSchema variantSchema) {
+    super(name, type, DataMode.OPTIONAL);
+    this.variantSchema = variantSchema == null ? new VariantSchema() : variantSchema;
+    this.variantSchema.bind(this);
+  }
+
+  @Override
+  public StructureType structureType() {
+    return StructureType.VARIANT;
+  }
+
+  @Override
+  public boolean isVariant() { return true; }
+
+  @Override
+  public boolean isArray() { return type() == MinorType.LIST; }
+
+  @Override
+  public ColumnMetadata cloneEmpty() {
+    return new VariantColumnMetadata(name, type, variantSchema.cloneEmpty());
+  }
+
+  @Override
+  public AbstractColumnMetadata copy() {
+    // TODO Auto-generated method stub
+    assert false;
+    return null;
+  }
+
+  @Override
+  public VariantMetadata variantSchema() {
+    return variantSchema;
+  }
+
+  @Override
+  public MaterializedField schema() {
+    return MaterializedField.create(name,
+        MajorType.newBuilder()
+          .setMinorType(type)
+          .setMode(DataMode.OPTIONAL)
+          .addAllSubType(variantSchema.types())
+          .build());
+  }
+
+  @Override
+  public MaterializedField emptySchema() {
+    return MaterializedField.create(name,
+        MajorType.newBuilder()
+          .setMinorType(type)
+          .setMode(DataMode.OPTIONAL)
+          .build());
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/VariantSchema.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/VariantSchema.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/VariantSchema.java
new file mode 100644
index 0000000..cfa8cee
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/VariantSchema.java
@@ -0,0 +1,210 @@
+/*
+ * 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.record.metadata;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.drill.common.types.Types;
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.record.MaterializedField;
+
+import com.google.common.base.Preconditions;
+
+public class VariantSchema implements VariantMetadata {
+
+  private final Map<MinorType, ColumnMetadata> types = new HashMap<>();
+  private VariantColumnMetadata parent;
+  private boolean isSimple;
+
+  protected void bind(VariantColumnMetadata parent) {
+    this.parent = parent;
+  }
+
+  public static AbstractColumnMetadata memberMetadata(MinorType type) {
+    String name = Types.typeKey(type);
+    switch (type) {
+    case LIST:
+      return new VariantColumnMetadata(name, type, null);
+    case MAP:
+      // Although maps do not have a bits vector, when used in a
+      // union the map must be marked as optional since the union as a
+      // whole can be null, implying that the map is null by implication.
+      // (In fact, the readers have a special mechanism to work out the
+      // null state in this case.
+
+      return new MapColumnMetadata(name, DataMode.OPTIONAL, null);
+    case UNION:
+      throw new IllegalArgumentException("Cannot add a union to a union");
+    default:
+      return new PrimitiveColumnMetadata(
+          MaterializedField.create(
+              name,
+              Types.optional(type)));
+    }
+  }
+
+  @Override
+  public ColumnMetadata addType(MinorType type) {
+    checkType(type);
+    AbstractColumnMetadata dummyCol = memberMetadata(type);
+    types.put(type, dummyCol);
+    return dummyCol;
+  }
+
+  @Override
+  public void addType(ColumnMetadata col) {
+    checkType(col.type());
+    Preconditions.checkArgument(col.name().equals(Types.typeKey(col.type())));
+    switch (col.type()) {
+    case UNION:
+      throw new IllegalArgumentException("Cannot add a union to a union");
+    case LIST:
+      if (col.mode() == DataMode.REQUIRED) {
+        throw new IllegalArgumentException("List type column must be OPTIONAL or REPEATED");
+      }
+      break;
+    default:
+      if (col.mode() != DataMode.OPTIONAL) {
+        throw new IllegalArgumentException("Type column must be OPTIONAL");
+      }
+      break;
+    }
+    types.put(col.type(), col);
+  }
+
+  private void checkType(MinorType type) {
+    if (types.containsKey(type)) {
+      throw new IllegalArgumentException("Variant already contains type: " + type);
+    }
+  }
+
+  @Override
+  public int size() { return types.size(); }
+
+  @Override
+  public boolean hasType(MinorType type) {
+    return types.containsKey(type);
+  }
+
+  @Override
+  public ColumnMetadata member(MinorType type) {
+    return types.get(type);
+  }
+
+  @Override
+  public ColumnMetadata parent() { return parent; }
+
+  @Override
+  public Collection<MinorType> types() {
+    return types.keySet();
+  }
+
+  @Override
+  public Collection<ColumnMetadata> members() {
+    return types.values();
+  }
+
+  public void addMap(MapColumnMetadata mapCol) {
+    Preconditions.checkArgument(! mapCol.isArray());
+    Preconditions.checkState(! isSimple);
+    checkType(MinorType.MAP);
+    types.put(MinorType.MAP, mapCol);
+  }
+
+  public void addList(VariantColumnMetadata listCol) {
+    Preconditions.checkArgument(listCol.isArray());
+    Preconditions.checkState(! isSimple);
+    checkType(MinorType.LIST);
+    types.put(MinorType.LIST, listCol);
+  }
+
+  public ColumnMetadata addType(MaterializedField field) {
+    Preconditions.checkState(! isSimple);
+    MinorType type = field.getType().getMinorType();
+    checkType(type);
+    AbstractColumnMetadata col;
+    switch (type) {
+    case LIST:
+      col = new VariantColumnMetadata(field);
+      break;
+    case MAP:
+      col = new MapColumnMetadata(field);
+      break;
+    case UNION:
+      throw new IllegalArgumentException("Cannot add a union to a union");
+    default:
+      col = new PrimitiveColumnMetadata(field);
+      break;
+    }
+    types.put(type, col);
+    return col;
+  }
+
+  @Override
+  public boolean isSingleType() {
+    return types.size() == 1;
+  }
+
+  @Override
+  public ColumnMetadata listSubtype() {
+    if (isSingleType()) {
+      return types.values().iterator().next();
+    }
+
+    // At the metadata level, a list always holds a union. But, at the
+    // implementation layer, a union of a single type is collapsed out
+    // to leave just a list of that single type.
+    //
+    // Make up a synthetic union column to be used when building
+    // a reader.
+
+    return new VariantColumnMetadata("$data", MinorType.UNION, this);
+  }
+
+  @Override
+  public void becomeSimple() {
+    Preconditions.checkState(types.size() == 1);
+    isSimple = true;
+  }
+
+  @Override
+  public boolean isSimple() {
+    return isSimple;
+  }
+
+  @Override
+  public String toString() {
+    return new StringBuilder()
+        .append("[")
+        .append(getClass().getSimpleName())
+        .append(types.toString())
+        .append(", simple: ")
+        .append(isSimple)
+        .append("]")
+        .toString();
+  }
+
+  public VariantSchema cloneEmpty() {
+    VariantSchema copy = new VariantSchema();
+    copy.isSimple = isSimple;
+    return copy;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/cf2478f7/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestExternalSort.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestExternalSort.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestExternalSort.java
index a79ecf5..9ef1976 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestExternalSort.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestExternalSort.java
@@ -17,14 +17,17 @@
  */
 package org.apache.drill.exec.physical.impl.xsort;
 
+import java.io.File;
+import java.nio.file.Paths;
+
+import org.apache.drill.categories.OperatorTest;
+import org.apache.drill.categories.SlowTest;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.test.BaseTestQuery;
 import org.apache.drill.test.TestBuilder;
-import org.apache.drill.categories.OperatorTest;
-import org.apache.drill.categories.SlowTest;
-import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.test.rowSet.RowSet;
 import org.apache.drill.test.rowSet.RowSetBuilder;
 import org.apache.drill.test.rowSet.SchemaBuilder;
@@ -33,9 +36,6 @@ import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import java.io.File;
-import java.nio.file.Paths;
-
 @Category({SlowTest.class, OperatorTest.class})
 public class TestExternalSort extends BaseTestQuery {
 
@@ -206,8 +206,8 @@ public class TestExternalSort extends BaseTestQuery {
 
     {
       final BatchSchema schema = new SchemaBuilder()
-        .add("a", Types.required(TypeProtos.MinorType.INT))
-        .add("b", Types.required(TypeProtos.MinorType.INT))
+        .add("a", TypeProtos.MinorType.INT)
+        .add("b", TypeProtos.MinorType.INT)
         .build();
       final RowSetBuilder rowSetBuilder = new RowSetBuilder(allocator, schema);
 
@@ -223,8 +223,8 @@ public class TestExternalSort extends BaseTestQuery {
 
     {
       final BatchSchema schema = new SchemaBuilder()
-        .add("a", Types.required(TypeProtos.MinorType.INT))
-        .add("c", Types.required(TypeProtos.MinorType.INT))
+        .add("a", TypeProtos.MinorType.INT)
+        .add("c", TypeProtos.MinorType.INT)
         .build();
       final RowSetBuilder rowSetBuilder = new RowSetBuilder(allocator, schema);
 


[07/17] drill git commit: DRILL-6198: OpenTSDB unit tests fail when Lilith client is running

Posted by ar...@apache.org.
DRILL-6198: OpenTSDB unit tests fail when Lilith client is running

closes #1142


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/4bd3cc29
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/4bd3cc29
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/4bd3cc29

Branch: refs/heads/master
Commit: 4bd3cc2995ab7bd94ca5f5a8838b0d02fb396c3e
Parents: 57e5ab2
Author: Volodymyr Vysotskyi <vv...@gmail.com>
Authored: Thu Mar 1 14:52:28 2018 +0200
Committer: Arina Ielchiieva <ar...@gmail.com>
Committed: Sat Mar 3 19:47:41 2018 +0200

----------------------------------------------------------------------
 .../store/openTSDB/TestOpenTSDBPlugin.java      | 41 +++++++++++++++++---
 1 file changed, 35 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/4bd3cc29/contrib/storage-opentsdb/src/test/java/org/apache/drill/store/openTSDB/TestOpenTSDBPlugin.java
----------------------------------------------------------------------
diff --git a/contrib/storage-opentsdb/src/test/java/org/apache/drill/store/openTSDB/TestOpenTSDBPlugin.java b/contrib/storage-opentsdb/src/test/java/org/apache/drill/store/openTSDB/TestOpenTSDBPlugin.java
index 27ca09c..0c1fb5e 100644
--- a/contrib/storage-opentsdb/src/test/java/org/apache/drill/store/openTSDB/TestOpenTSDBPlugin.java
+++ b/contrib/storage-opentsdb/src/test/java/org/apache/drill/store/openTSDB/TestOpenTSDBPlugin.java
@@ -21,13 +21,18 @@ import com.github.tomakehurst.wiremock.junit.WireMockRule;
 import org.apache.drill.PlanTestBase;
 import org.apache.drill.common.exceptions.UserRemoteException;
 import org.apache.drill.exec.store.StoragePluginRegistry;
-import org.apache.drill.exec.store.openTSDB.OpenTSDBStoragePlugin;
 import org.apache.drill.exec.store.openTSDB.OpenTSDBStoragePluginConfig;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.BindException;
+import java.net.ServerSocket;
 
 import static com.github.tomakehurst.wiremock.client.WireMock.aResponse;
 import static com.github.tomakehurst.wiremock.client.WireMock.equalToJson;
@@ -50,18 +55,19 @@ import static org.apache.drill.store.openTSDB.TestDataHolder.SAMPLE_DATA_FOR_POS
 import static org.apache.drill.store.openTSDB.TestDataHolder.SAMPLE_DATA_FOR_POST_REQUEST_WITH_TAGS;
 
 public class TestOpenTSDBPlugin extends PlanTestBase {
+  private static final Logger logger = LoggerFactory.getLogger(TestOpenTSDBPlugin.class);
 
-  protected static OpenTSDBStoragePlugin storagePlugin;
-  protected static OpenTSDBStoragePluginConfig storagePluginConfig;
+  private static int portNumber;
 
   @Rule
-  public WireMockRule wireMockRule = new WireMockRule(10000);
+  public WireMockRule wireMockRule = new WireMockRule(portNumber);
 
   @BeforeClass
   public static void setup() throws Exception {
+    portNumber = getFreePortNumber(10_000, 200);
     final StoragePluginRegistry pluginRegistry = getDrillbitContext().getStorage();
-    storagePlugin = (OpenTSDBStoragePlugin) pluginRegistry.getPlugin(OpenTSDBStoragePluginConfig.NAME);
-    storagePluginConfig = storagePlugin.getConfig();
+    OpenTSDBStoragePluginConfig storagePluginConfig =
+        new OpenTSDBStoragePluginConfig(String.format("http://localhost:%s", portNumber));
     storagePluginConfig.setEnabled(true);
     pluginRegistry.createOrUpdate(OpenTSDBStoragePluginConfig.NAME, storagePluginConfig, true);
   }
@@ -185,4 +191,27 @@ public class TestOpenTSDBPlugin extends PlanTestBase {
     test("describe `warp.speed.test`");
     Assert.assertEquals(1, testSql("show tables"));
   }
+
+  /**
+   * Checks that port with specified number is free and returns it.
+   * Otherwise, increases port number and checks until free port is found
+   * or the number of attempts is reached specified numberOfAttempts
+   *
+   * @param portNumber     initial port number
+   * @param numberOfAttempts max number of attempts to find port with greater number
+   * @return free port number
+   * @throws BindException if free port was not found and all attempts were used.
+   */
+  private static int getFreePortNumber(int portNumber, int numberOfAttempts) throws IOException {
+    for (int i = portNumber; i <= portNumber + numberOfAttempts; i++) {
+      try (ServerSocket socket = new ServerSocket(i)) {
+        return socket.getLocalPort();
+      } catch (BindException e) {
+        logger.warn("Port {} is already in use.", i);
+      }
+    }
+
+    throw new BindException(String.format("Free port could not be found in the range [%s-%s].\n" +
+        "Please release any of used ports in this range.", portNumber, portNumber + numberOfAttempts));
+  }
 }


[08/17] drill git commit: DRILL-6191: Add acknowledgement sequence number and flags fields, details for flags

Posted by ar...@apache.org.
DRILL-6191: Add acknowledgement sequence number and flags fields, details for flags

closes #1134


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/40894225
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/40894225
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/40894225

Branch: refs/heads/master
Commit: 408942259800d9987f4e84b3cdbd47e29920e934
Parents: 4bd3cc2
Author: Ted Dunning <te...@gmail.com>
Authored: Tue Jan 2 16:20:35 2018 -0800
Committer: Arina Ielchiieva <ar...@gmail.com>
Committed: Sat Mar 3 19:47:43 2018 +0200

----------------------------------------------------------------------
 .../drill/exec/store/pcap/PcapDrillTable.java   |   2 +
 .../drill/exec/store/pcap/PcapRecordReader.java |  75 ++++++++++++++
 .../drill/exec/store/pcap/decoder/Packet.java   | 102 ++++++++++++++++++-
 .../store/pcap/decoder/PacketConstants.java     |   3 +
 .../drill/exec/store/pcap/schema/PcapTypes.java |   1 +
 .../drill/exec/store/pcap/schema/Schema.java    |  14 +++
 .../exec/store/pcap/TestPcapRecordReader.java   |  40 ++++++--
 .../src/test/resources/store/pcap/synscan.pcap  | Bin 0 -> 148872 bytes
 8 files changed, 224 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/40894225/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapDrillTable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapDrillTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapDrillTable.java
index 2fbf67d..20e7e93 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapDrillTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapDrillTable.java
@@ -53,6 +53,8 @@ public class PcapDrillTable extends DrillTable {
         return typeFactory.createSqlType(SqlTypeName.BIGINT);
       case INTEGER:
         return typeFactory.createSqlType(SqlTypeName.INTEGER);
+      case BOOLEAN:
+        return typeFactory.createSqlType(SqlTypeName.BOOLEAN);
       case STRING:
         return typeFactory.createSqlType(SqlTypeName.VARCHAR);
       case TIMESTAMP:

http://git-wip-us.apache.org/repos/asf/drill/blob/40894225/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapRecordReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapRecordReader.java
index 26e1e65..d01b746 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapRecordReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapRecordReader.java
@@ -280,6 +280,76 @@ public class PcapRecordReader extends AbstractRecordReader {
             setIntegerColumnValue(packet.getSequenceNumber(), pci, count);
           }
           break;
+        case "tcp_ack":
+          if (packet.isTcpPacket()) {
+            setIntegerColumnValue(packet.getAckNumber(), pci, count);
+          }
+          break;
+        case "tcp_flags":
+          if (packet.isTcpPacket()) {
+            setIntegerColumnValue(packet.getFlags(), pci, count);
+          }
+          break;
+        case "tcp_parsed_flags":
+          if (packet.isTcpPacket()) {
+            setStringColumnValue(packet.getParsedFlags(), pci, count);
+          }
+          break;
+        case "tcp_flags_ns":
+          if (packet.isTcpPacket()) {
+            setBooleanColumnValue((packet.getFlags() & 0x100) != 0, pci, count);
+          }
+          break;
+        case "tcp_flags_cwr":
+          if (packet.isTcpPacket()) {
+            setBooleanColumnValue((packet.getFlags() & 0x80) != 0, pci, count);
+          }
+          break;
+        case "tcp_flags_ece ":
+          if (packet.isTcpPacket()) {
+            setBooleanColumnValue((packet.getFlags() & 0x40) != 0, pci, count);
+          }
+          break;
+        case "tcp_flags_ece_ecn_capable":
+          if (packet.isTcpPacket()) {
+            setBooleanColumnValue((packet.getFlags() & 0x42) == 0x42, pci, count);
+          }
+          break;
+        case "tcp_flags_ece_congestion_experienced":
+          if (packet.isTcpPacket()) {
+            setBooleanColumnValue((packet.getFlags() & 0x42) == 0x40, pci, count);
+          }
+          break;
+        case "tcp_flags_urg":
+          if (packet.isTcpPacket()) {
+            setBooleanColumnValue((packet.getFlags() & 0x20) != 0, pci, count);
+          }
+          break;
+        case "tcp_flags_ack":
+          if (packet.isTcpPacket()) {
+            setBooleanColumnValue((packet.getFlags() & 0x10) != 0, pci, count);
+          }
+          break;
+        case "tcp_flags_psh":
+          if (packet.isTcpPacket()) {
+            setBooleanColumnValue((packet.getFlags() & 0x8) != 0, pci, count);
+          }
+          break;
+        case "tcp_flags_rst":
+          if (packet.isTcpPacket()) {
+            setBooleanColumnValue((packet.getFlags() & 0x4) != 0, pci, count);
+          }
+          break;
+        case "tcp_flags_syn":
+          if (packet.isTcpPacket()) {
+            setBooleanColumnValue((packet.getFlags() & 0x2) != 0, pci, count);
+          }
+          break;
+        case "tcp_flags_fin":
+          if (packet.isTcpPacket()) {
+            setBooleanColumnValue((packet.getFlags() & 0x1) != 0, pci, count);
+          }
+          break;
         case "packet_length":
           setIntegerColumnValue(packet.getPacketLength(), pci, count);
           break;
@@ -305,6 +375,11 @@ public class PcapRecordReader extends AbstractRecordReader {
         .setSafe(count, data);
   }
 
+  private void setBooleanColumnValue(final boolean data, final ProjectedColumnInfo pci, final int count) {
+    ((NullableIntVector.Mutator) pci.vv.getMutator())
+        .setSafe(count, data ? 1 : 0);
+  }
+
   private void setTimestampColumnValue(final long data, final ProjectedColumnInfo pci, final int count) {
     ((NullableTimeStampVector.Mutator) pci.vv.getMutator())
         .setSafe(count, data);

http://git-wip-us.apache.org/repos/asf/drill/blob/40894225/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/decoder/Packet.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/decoder/Packet.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/decoder/Packet.java
index 0a45290..9cc98de 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/decoder/Packet.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/decoder/Packet.java
@@ -23,6 +23,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
+import java.util.Formatter;
 
 import static org.apache.drill.exec.store.pcap.PcapFormatUtils.convertInt;
 import static org.apache.drill.exec.store.pcap.PcapFormatUtils.convertShort;
@@ -43,7 +44,9 @@ public class Packet {
 
   private byte[] raw;
 
+  // index into the raw data where the current ethernet packet starts
   private int etherOffset;
+  // index into the raw data where the current IP packet starts. Should be just after etherOffset
   private int ipOffset;
 
   private int packetLength;
@@ -180,13 +183,104 @@ public class Packet {
 
   public int getSequenceNumber() {
     if (isTcpPacket()) {
-      int sequenceOffset = PacketConstants.ETHER_HEADER_LENGTH + getIPHeaderLength() + getTCPHeaderLength(raw) + 4;
-      return Math.abs(convertInt(raw, sequenceOffset));
+      return convertInt(raw, ipOffset + getIPHeaderLength() + PacketConstants.TCP_SEQUENCE_OFFSET);
     } else {
       return 0;
     }
   }
 
+  public int getAckNumber() {
+    if (isTcpPacket()) {
+        return convertInt(raw, ipOffset + getIPHeaderLength() + PacketConstants.TCP_ACK_OFFSET);
+    } else {
+      return 0;
+    }
+  }
+
+  public int getFlags() {
+    if (isTcpPacket()) {
+      return convertShort(raw, ipOffset + getIPHeaderLength() + PacketConstants.TCP_FLAG_OFFSET) & 0xfff;
+    } else {
+      return 0;
+    }
+  }
+
+  public String getParsedFlags() {
+    return formatFlags(getFlags());
+  }
+
+  public static String formatFlags(int flags) {
+    int mask = 0x100;
+    StringBuilder r = new StringBuilder();
+    String separator = "";
+    if ((flags & mask) != 0) {
+      r.append(separator);
+      r.append("NS");
+      separator = "|";
+    }
+    mask = mask >> 1;
+
+    if ((flags & mask) != 0) {
+      r.append(separator);
+      r.append("CWR");
+      separator = "|";
+    }
+    mask = mask >> 1;
+
+    if ((flags & mask) != 0) {
+      r.append(separator);
+      r.append("ECE");
+      if ((flags & 2) != 0) {
+        r.append(" (ECN capable)");
+      } else {
+        r.append(" (Congestion experienced)");
+      }
+      separator = "|";
+    }
+    mask = mask >> 1;
+
+    if ((flags & mask) != 0) {
+      r.append(separator);
+      r.append("URG");
+      separator = "|";
+    }
+    mask = mask >> 1;
+
+    if ((flags & mask) != 0) {
+      r.append(separator);
+      r.append("ACK");
+      separator = "|";
+    }
+    mask = mask >> 1;
+
+    if ((flags & mask) != 0) {
+      r.append(separator);
+      r.append("PSH");
+      separator = "|";
+    }
+    mask = mask >> 1;
+
+    if ((flags & mask) != 0) {
+      r.append(separator);
+      r.append("RST");
+      separator = "|";
+    }
+    mask = mask >> 1;
+
+    if ((flags & mask) != 0) {
+      r.append(separator);
+      r.append("SYN");
+      separator = "|";
+    }
+    mask = mask >> 1;
+
+    if ((flags & mask) != 0) {
+      r.append(separator);
+      r.append("FIN");
+    }
+    return r.toString();
+  }
+
   public int getSrc_port() {
     if (isPPPoV6Packet()) {
       return getPort(64);
@@ -361,9 +455,9 @@ public class Packet {
   private String getEthernetAddress(int offset) {
     byte[] r = new byte[6];
     System.arraycopy(raw, etherOffset + offset, r, 0, 6);
-    StringBuilder sb = new StringBuilder();
+    Formatter sb = new Formatter();
     for (int i = 0; i < r.length; i++) {
-      sb.append(String.format("%02X%s", r[i], (i < r.length - 1) ? ":" : ""));
+      sb.format("%02X%s", r[i], (i < r.length - 1) ? ":" : "");
     }
     return sb.toString();
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/40894225/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/decoder/PacketConstants.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/decoder/PacketConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/decoder/PacketConstants.java
index 2c87623..6f29253 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/decoder/PacketConstants.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/decoder/PacketConstants.java
@@ -65,4 +65,7 @@ public final class PacketConstants {
 
   public static final int PPPoV6_IP_OFFSET = 28;
 
+  public static final int TCP_SEQUENCE_OFFSET = 4;
+  public static final int TCP_ACK_OFFSET = 8;
+  public static final int TCP_FLAG_OFFSET = 12;
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/40894225/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/schema/PcapTypes.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/schema/PcapTypes.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/schema/PcapTypes.java
index 5c6df71..fc6e029 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/schema/PcapTypes.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/schema/PcapTypes.java
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.store.pcap.schema;
 
 public enum PcapTypes {
+  BOOLEAN,
   INTEGER,
   STRING,
   LONG,

http://git-wip-us.apache.org/repos/asf/drill/blob/40894225/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/schema/Schema.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/schema/Schema.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/schema/Schema.java
index b3e7722..89bd08f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/schema/Schema.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/schema/Schema.java
@@ -41,6 +41,20 @@ public class Schema {
     columns.add(new ColumnDto("src_mac_address", PcapTypes.STRING));
     columns.add(new ColumnDto("dst_mac_address", PcapTypes.STRING));
     columns.add(new ColumnDto("tcp_session", PcapTypes.LONG));
+    columns.add(new ColumnDto("tcp_ack", PcapTypes.INTEGER));
+    columns.add(new ColumnDto("tcp_flags", PcapTypes.INTEGER));
+    columns.add(new ColumnDto("tcp_flags_ns", PcapTypes.INTEGER));
+    columns.add(new ColumnDto("tcp_flags_cwr", PcapTypes.INTEGER));
+    columns.add(new ColumnDto("tcp_flags_ece ", PcapTypes.INTEGER ));
+    columns.add(new ColumnDto("tcp_flags_ece_ecn_capable", PcapTypes.INTEGER ));
+    columns.add(new ColumnDto("tcp_flags_ece_congestion_experienced", PcapTypes.INTEGER ));
+    columns.add(new ColumnDto("tcp_flags_urg", PcapTypes.INTEGER ));
+    columns.add(new ColumnDto("tcp_flags_ack", PcapTypes.INTEGER ));
+    columns.add(new ColumnDto("tcp_flags_psh", PcapTypes.INTEGER ));
+    columns.add(new ColumnDto("tcp_flags_rst", PcapTypes.INTEGER ));
+    columns.add(new ColumnDto("tcp_flags_syn", PcapTypes.INTEGER ));
+    columns.add(new ColumnDto("tcp_flags_fin", PcapTypes.INTEGER ));
+    columns.add(new ColumnDto("tcp_parsed_flags", PcapTypes.STRING));
     columns.add(new ColumnDto("packet_length", PcapTypes.INTEGER));
     columns.add(new ColumnDto("data", PcapTypes.STRING));
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/40894225/exec/java-exec/src/test/java/org/apache/drill/exec/store/pcap/TestPcapRecordReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/pcap/TestPcapRecordReader.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/pcap/TestPcapRecordReader.java
index bb81469..385c0e0 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/pcap/TestPcapRecordReader.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/pcap/TestPcapRecordReader.java
@@ -16,16 +16,17 @@
  */
 package org.apache.drill.exec.store.pcap;
 
+import org.apache.drill.exec.store.pcap.decoder.Packet;
 import org.apache.drill.test.BaseTestQuery;
-import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.rpc.user.QueryDataBatch;
-import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 import java.nio.file.Paths;
 import java.util.List;
 
+import static org.junit.Assert.assertEquals;
+
 public class TestPcapRecordReader extends BaseTestQuery {
   @BeforeClass
   public static void setupTestFiles() {
@@ -52,21 +53,42 @@ public class TestPcapRecordReader extends BaseTestQuery {
     runSQLVerifyCount("select distinct type, network, `timestamp`, src_ip, dst_ip, src_port, dst_port, src_mac_address, dst_mac_address, tcp_session, packet_length from dfs.`store/pcap/tcp-1.pcap`", 1);
   }
 
+  @Test
+  public void testFlagFormatting() {
+    assertEquals("NS", Packet.formatFlags(0x100));
+    assertEquals("CWR", Packet.formatFlags(0x80));
+    assertEquals("ECE", Packet.formatFlags(0x40).substring(0, 3));
+    assertEquals("ECE", Packet.formatFlags(0x42).substring(0, 3));
+    assertEquals("URG", Packet.formatFlags(0x20));
+    assertEquals("ACK", Packet.formatFlags(0x10));
+    assertEquals("PSH", Packet.formatFlags(0x8));
+    assertEquals("RST", Packet.formatFlags(0x4));
+    assertEquals("SYN", Packet.formatFlags(0x2));
+    assertEquals("FIN", Packet.formatFlags(0x1));
+    assertEquals("RST|SYN|FIN", Packet.formatFlags(0x7));
+  }
+
+  @Test
+  public void checkFlags() throws Exception {
+    runSQLVerifyCount("select tcp_session, tcp_ack, tcp_flags from dfs.`store/pcap/synscan.pcap`", 2011);
+  }
+
   private void runSQLVerifyCount(String sql, int expectedRowCount) throws Exception {
     List<QueryDataBatch> results = runSQLWithResults(sql);
-    printResultAndVerifyRowCount(results, expectedRowCount);
+    verifyRowCount(results, expectedRowCount);
   }
 
   private List<QueryDataBatch> runSQLWithResults(String sql) throws Exception {
     return testSqlWithResults(sql);
   }
 
-  private void printResultAndVerifyRowCount(List<QueryDataBatch> results,
-                                            int expectedRowCount) throws SchemaChangeException {
-    setColumnWidth(35);
-    int rowCount = printResult(results);
-    if (expectedRowCount != -1) {
-      Assert.assertEquals(expectedRowCount, rowCount);
+  private void verifyRowCount(List<QueryDataBatch> results, int expectedRowCount) {
+    int count = 0;
+    for (final QueryDataBatch result : results) {
+      count += result.getHeader().getRowCount();
+      result.release();
     }
+    assertEquals(expectedRowCount, count);
   }
+
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/40894225/exec/java-exec/src/test/resources/store/pcap/synscan.pcap
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/store/pcap/synscan.pcap b/exec/java-exec/src/test/resources/store/pcap/synscan.pcap
new file mode 100644
index 0000000..8c2ca36
Binary files /dev/null and b/exec/java-exec/src/test/resources/store/pcap/synscan.pcap differ


[14/17] drill git commit: DRILL-6153: Operator framework

Posted by ar...@apache.org.
DRILL-6153: Operator framework

closes #1121


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/69a5f3a9
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/69a5f3a9
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/69a5f3a9

Branch: refs/heads/master
Commit: 69a5f3a9c4fadafc588a3e325a12b98cbf359ece
Parents: 4ee207b
Author: Paul Rogers <pr...@cloudera.com>
Authored: Mon Feb 12 22:27:23 2018 -0800
Committer: Arina Ielchiieva <ar...@gmail.com>
Committed: Sat Mar 3 19:47:53 2018 +0200

----------------------------------------------------------------------
 .../physical/impl/protocol/BatchAccessor.java   |  50 ++
 .../physical/impl/protocol/OperatorDriver.java  | 234 ++++++
 .../physical/impl/protocol/OperatorExec.java    | 127 ++++
 .../impl/protocol/OperatorRecordBatch.java      | 156 ++++
 .../physical/impl/protocol/SchemaTracker.java   |  98 +++
 .../impl/protocol/VectorContainerAccessor.java  | 134 ++++
 .../physical/impl/protocol/package-info.java    |  29 +
 .../impl/protocol/TestOperatorRecordBatch.java  | 747 +++++++++++++++++++
 8 files changed, 1575 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/69a5f3a9/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/protocol/BatchAccessor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/protocol/BatchAccessor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/protocol/BatchAccessor.java
new file mode 100644
index 0000000..b22353f
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/protocol/BatchAccessor.java
@@ -0,0 +1,50 @@
+/*
+ * 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.protocol;
+
+import java.util.Iterator;
+
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.TypedFieldId;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.record.WritableBatch;
+import org.apache.drill.exec.record.selection.SelectionVector2;
+import org.apache.drill.exec.record.selection.SelectionVector4;
+
+/**
+ * Provides access to the row set (record batch) produced by an
+ * operator. Previously, a record batch <i>was</i> an operator.
+ * In this version, the row set is a service of the operator rather
+ * than being part of the operator.
+ */
+
+public interface BatchAccessor {
+  BatchSchema getSchema();
+  int schemaVersion();
+  int getRowCount();
+  VectorContainer getOutgoingContainer();
+  TypedFieldId getValueVectorId(SchemaPath path);
+  VectorWrapper<?> getValueAccessorById(Class<?> clazz, int... ids);
+  WritableBatch getWritableBatch();
+  SelectionVector2 getSelectionVector2();
+  SelectionVector4 getSelectionVector4();
+  Iterator<VectorWrapper<?>> iterator();
+  void release();
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/69a5f3a9/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/protocol/OperatorDriver.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/protocol/OperatorDriver.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/protocol/OperatorDriver.java
new file mode 100644
index 0000000..9e6190c
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/protocol/OperatorDriver.java
@@ -0,0 +1,234 @@
+/*
+ * 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.protocol;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.record.RecordBatch.IterOutcome;
+
+/**
+ * State machine that drives the operator executable. Converts
+ * between the iterator protocol and the operator executable protocol.
+ * Implemented as a separate class in anticipation of eventually
+ * changing the record batch (iterator) protocol.
+ */
+
+public class OperatorDriver {
+  public enum State {
+
+    /**
+     * Before the first call to next().
+     */
+
+    START,
+
+    /**
+     * The first call to next() has been made and schema (only)
+     * was returned. On the subsequent call to next(), return any
+     * data that might have accompanied that first batch.
+     */
+
+    SCHEMA,
+
+    /**
+     * The second call to next() has been made and there is more
+     * data to deliver on subsequent calls.
+     */
+
+    RUN,
+
+    /**
+     * No more data to deliver.
+     */
+
+    END,
+
+    /**
+     * An error occurred.
+     */
+
+    FAILED,
+
+    /**
+     * Operation was cancelled. No more batches will be returned,
+     * but close() has not yet been called.
+     */
+
+    CANCELED,
+
+    /**
+     * close() called and resources are released. No more batches
+     * will be returned, but close() has not yet been called.
+     * (This state is semantically identical to FAILED, it exists just
+     * in case an implementation needs to know the difference between the
+     * END, FAILED and CANCELED states.)
+     */
+
+    CLOSED
+  }
+
+  private OperatorDriver.State state = State.START;
+
+  /**
+   * Operator context. The driver "owns" the context and is responsible
+   * for closing it.
+   */
+
+  private final OperatorContext opContext;
+  private final OperatorExec operatorExec;
+  private final BatchAccessor batchAccessor;
+  private int schemaVersion;
+
+  public OperatorDriver(OperatorContext opContext, OperatorExec opExec) {
+    this.opContext = opContext;
+    this.operatorExec = opExec;
+    batchAccessor = operatorExec.batchAccessor();
+  }
+
+  /**
+   * Get the next batch. Performs initialization on the first call.
+   * @return the iteration outcome to send downstream
+   */
+
+  public IterOutcome next() {
+    try {
+      switch (state) {
+      case START:
+        return start();
+      case RUN:
+        return doNext();
+      default:
+        OperatorRecordBatch.logger.debug("Extra call to next() in state " + state + ": " + operatorLabel());
+        return IterOutcome.NONE;
+      }
+    } catch (UserException e) {
+      cancelSilently();
+      state = State.FAILED;
+      throw e;
+    } catch (Throwable t) {
+      cancelSilently();
+      state = State.FAILED;
+      throw UserException.executionError(t)
+        .addContext("Exception thrown from", operatorLabel())
+        .build(OperatorRecordBatch.logger);
+    }
+  }
+
+  /**
+   * Cancels the operator before reaching EOF.
+   */
+
+  public void cancel() {
+    try {
+      switch (state) {
+      case START:
+      case RUN:
+        cancelSilently();
+        break;
+      default:
+        break;
+      }
+    } finally {
+      state = State.CANCELED;
+    }
+  }
+
+ /**
+   * Start the operator executor. Bind it to the various contexts.
+   * Then start the executor and fetch the first schema.
+   * @return result of the first batch, which should contain
+   * only a schema, or EOF
+   */
+
+  private IterOutcome start() {
+    state = State.SCHEMA;
+    if (operatorExec.buildSchema()) {
+      schemaVersion = batchAccessor.schemaVersion();
+      state = State.RUN;
+      return IterOutcome.OK_NEW_SCHEMA;
+    } else {
+      state = State.END;
+      return IterOutcome.NONE;
+    }
+  }
+
+  /**
+   * Fetch a record batch, detecting EOF and a new schema.
+   * @return the <tt>IterOutcome</tt> for the above cases
+   */
+
+  private IterOutcome doNext() {
+    if (! operatorExec.next()) {
+      state = State.END;
+      return IterOutcome.NONE;
+    }
+    int newVersion = batchAccessor.schemaVersion();
+    if (newVersion != schemaVersion) {
+      schemaVersion = newVersion;
+      return IterOutcome.OK_NEW_SCHEMA;
+    }
+    return IterOutcome.OK;
+  }
+
+  /**
+   * Implement a cancellation, and ignore any exception that is
+   * thrown. We're already in trouble here, no need to keep track
+   * of additional things that go wrong.
+   */
+
+  private void cancelSilently() {
+    try {
+      if (state == State.SCHEMA || state == State.RUN) {
+        operatorExec.cancel();
+      }
+    } catch (Throwable t) {
+      // Ignore; we're already in a bad state.
+      OperatorRecordBatch.logger.error("Exception thrown from cancel() for " + operatorLabel(), t);
+    }
+  }
+
+  private String operatorLabel() {
+    return operatorExec.getClass().getCanonicalName();
+  }
+
+  public void close() {
+    if (state == State.CLOSED) {
+      return;
+    }
+    try {
+      operatorExec.close();
+    } catch (UserException e) {
+      throw e;
+    } catch (Throwable t) {
+      throw UserException.executionError(t)
+        .addContext("Exception thrown from", operatorLabel())
+        .build(OperatorRecordBatch.logger);
+    } finally {
+      opContext.close();
+      state = State.CLOSED;
+    }
+  }
+
+  public BatchAccessor batchAccessor() {
+    return batchAccessor;
+  }
+
+  public OperatorContext operatorContext() {
+    return opContext;
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/69a5f3a9/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/protocol/OperatorExec.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/protocol/OperatorExec.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/protocol/OperatorExec.java
new file mode 100644
index 0000000..57a8cf3
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/protocol/OperatorExec.java
@@ -0,0 +1,127 @@
+/*
+ * 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.protocol;
+
+import org.apache.drill.exec.ops.OperatorContext;
+
+/**
+ * Core protocol for a Drill operator execution.
+ *
+ * <h4>Lifecycle</h4>
+ *
+ * <ul>
+ * <li>Creation via an operator-specific constructor in the
+ * corresponding <tt>RecordBatchCreator</tt>.</li>
+ * <li><tt>bind()</tt> called to provide the operator services.</li>
+ * <li><tt>buildSchema()</tt> called to define the schema before
+ * fetching the first record batch.</li>
+ * <li><tt>next()</tt> called repeatedly to prepare each new record
+ * batch until EOF or until cancellation.</li>
+ * <li><tt>cancel()</tt> called if the operator should quit early.</li>
+ * <li><tt>close()</tt> called to release resources. Note that
+ * <tt>close()</tt> is called in response to:<ul>
+ *   <li>EOF</li>
+ *   <li>After <tt>cancel()</tt></li>
+ *   <li>After an exception is thrown.</li></ul></li>
+ * </ul>
+ *
+ * <h4>Error Handling</h4>
+ *
+ * Any method can throw an (unchecked) exception. (Drill does not use
+ * checked exceptions.) Preferably, the code will throw a
+ * <tt>UserException</tt> that explains the error to the user. If any
+ * other kind of exception is thrown, then the enclosing class wraps it
+ * in a generic <tt>UserException</tt> that indicates that "something went
+ * wrong", which is less than ideal.
+ *
+ * <h4>Result Set</h4>
+ * The operator "publishes" a result set in response to returning
+ * <tt>true</tt> from <tt>next()</tt> by populating a
+ * {@link BatchAccesor} provided via {@link #batchAccessor()}. For
+ * compatibility with other Drill operators, the set of vectors within
+ * the batch must be the same from one batch to the next.
+ */
+
+public interface OperatorExec {
+
+  /**
+   * Bind this operator to the context. The context provides access
+   * to per-operator, per-fragment and per-Drillbit services.
+   * Also provides access to the operator definition (AKA "pop
+   * config") for this operator.
+   *
+   * @param context operator context
+   */
+
+  public void bind(OperatorContext context);
+
+  /**
+   * Provides a generic access mechanism to the batch's output data.
+   * This method is called after a successful return from
+   * {@link #buildSchema()} and {@link #next()}. The batch itself
+   * can be held in a standard {@link VectorContainer}, or in some
+   * other structure more convenient for this operator.
+   *
+   * @return the access for the batch's output container
+   */
+
+  BatchAccessor batchAccessor();
+
+  /**
+   * Retrieves the schema of the batch before the first actual batch
+   * of data. The schema is returned via an empty batch (no rows,
+   * only schema) from {@link #batchAccessor()}.
+   *
+   * @return true if a schema is available, false if the operator
+   * reached EOF before a schema was found
+   */
+
+  boolean buildSchema();
+
+  /**
+   * Retrieves the next batch of data. The data is returned via
+   * the {@link #batchAccessor()} method.
+   *
+   * @return true if another batch of data is available, false if
+   * EOF was reached and no more data is available
+   */
+
+  boolean next();
+
+  /**
+   * Alerts the operator that the query was cancelled. Generally
+   * optional, but allows the operator to realize that a cancellation
+   * was requested.
+   */
+
+  void cancel();
+
+  /**
+   * Close the operator by releasing all resources that the operator
+   * held. Called after {@link #cancel()} and after {@link #batchAccessor()}
+   * or {@link #next()} returns false.
+   * <p>
+   * Note that there may be a significant delay between the last call to
+   * <tt>next()</tt> and the call to <tt>close()</tt> during which
+   * downstream operators do their work. A tidy operator will release
+   * resources immediately after EOF to avoid holding onto memory or other
+   * resources that could be used by downstream operators.
+   */
+
+  void close();
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/69a5f3a9/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/protocol/OperatorRecordBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/protocol/OperatorRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/protocol/OperatorRecordBatch.java
new file mode 100644
index 0000000..4f0cff8
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/protocol/OperatorRecordBatch.java
@@ -0,0 +1,156 @@
+/*
+ * 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.protocol;
+
+import java.util.Iterator;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.CloseableRecordBatch;
+import org.apache.drill.exec.record.TypedFieldId;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.record.WritableBatch;
+import org.apache.drill.exec.record.selection.SelectionVector2;
+import org.apache.drill.exec.record.selection.SelectionVector4;
+
+/**
+ * Modular implementation of the standard Drill record batch iterator
+ * protocol. The protocol has two parts: control of the operator and
+ * access to the record batch. Each is encapsulated in separate
+ * implementation classes to allow easier customization for each
+ * situation. The operator internals are, themselves, abstracted to
+ * yet another class with the steps represented as method calls rather
+ * than as internal states as in the record batch iterator protocol.
+ * <p>
+ * Note that downstream operators make an assumption that the
+ * same vectors will appear from one batch to the next. That is,
+ * not only must the schema be the same, but if column "a" appears
+ * in two batches, the same value vector must back "a" in both
+ * batches. The <tt>TransferPair</tt> abstraction fails if different
+ * vectors appear across batches.
+ */
+
+public class OperatorRecordBatch implements CloseableRecordBatch {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(OperatorRecordBatch.class);
+
+  private final OperatorDriver driver;
+  private final BatchAccessor batchAccessor;
+
+  public OperatorRecordBatch(FragmentContext context, PhysicalOperator config, OperatorExec opExec) {
+    OperatorContext opContext = context.newOperatorContext(config);
+    opContext.getStats().startProcessing();
+
+    // Chicken-and-egg binding: the two objects must know about each other. Pass the
+    // context to the operator exec via a bind method.
+
+    try {
+      opExec.bind(opContext);
+      driver = new OperatorDriver(opContext, opExec);
+      batchAccessor = opExec.batchAccessor();
+    } catch (UserException e) {
+      opContext.close();
+      throw e;
+    } catch (Throwable t) {
+      opContext.close();
+      throw UserException.executionError(t)
+        .addContext("Exception thrown from", opExec.getClass().getSimpleName() + ".bind()")
+        .build(logger);
+    }
+    finally {
+      opContext.getStats().stopProcessing();
+    }
+  }
+
+  @Override
+  public FragmentContext getContext() {
+    return fragmentContext();
+  }
+
+  // No longer needed, can be removed after all
+  // batch size control work is committed.
+
+  public FragmentContext fragmentContext() {
+    return driver.operatorContext().getFragmentContext();
+  }
+
+  @Override
+  public BatchSchema getSchema() { return batchAccessor.getSchema(); }
+
+  @Override
+  public int getRecordCount() { return batchAccessor.getRowCount(); }
+
+  @Override
+  public VectorContainer getOutgoingContainer() {
+    return batchAccessor.getOutgoingContainer();
+  }
+
+  @Override
+  public TypedFieldId getValueVectorId(SchemaPath path) {
+    return batchAccessor.getValueVectorId(path);
+  }
+
+  @Override
+  public VectorWrapper<?> getValueAccessorById(Class<?> clazz, int... ids) {
+    return batchAccessor.getValueAccessorById(clazz, ids);
+  }
+
+  @Override
+  public WritableBatch getWritableBatch() {
+    return batchAccessor.getWritableBatch();
+  }
+
+  @Override
+  public SelectionVector2 getSelectionVector2() {
+    return batchAccessor.getSelectionVector2();
+  }
+
+  @Override
+  public SelectionVector4 getSelectionVector4() {
+    return batchAccessor.getSelectionVector4();
+  }
+
+  @Override
+  public Iterator<VectorWrapper<?>> iterator() {
+    return batchAccessor.iterator();
+  }
+
+  @Override
+  public void kill(boolean sendUpstream) {
+    driver.cancel();
+  }
+
+  @Override
+  public IterOutcome next() {
+    try {
+      driver.operatorContext().getStats().startProcessing();
+      return driver.next();
+    } finally {
+      driver.operatorContext().getStats().stopProcessing();
+    }
+  }
+
+  @Override
+  public void close() {
+    driver.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/69a5f3a9/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/protocol/SchemaTracker.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/protocol/SchemaTracker.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/protocol/SchemaTracker.java
new file mode 100644
index 0000000..cd7c296
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/protocol/SchemaTracker.java
@@ -0,0 +1,98 @@
+/*
+ * 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.protocol;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.vector.ValueVector;
+
+/**
+ * Tracks changes to schemas via "snapshots" over time. That is, given
+ * a schema, tracks if a new schema is the same as the current one. For
+ * example, each batch output from a series of readers might be compared,
+ * as they are returned, to detect schema changes from one batch to the
+ * next. This class does not track vector-by-vector changes as a schema
+ * is built, but rather periodic "snapshots" at times determined by the
+ * operator.
+ * <p>
+ * If an operator is guaranteed to emit a consistent schema, then no
+ * checks need be done, and this tracker will report no schema change.
+ * On the other hand, a scanner might check schema more often. At least
+ * once per reader, and more often if a reader is "late-schema": if the
+ * reader can change schema batch-by-batch.
+ * <p>
+ * Drill defines "schema change" in a very specific way. Not only must
+ * the set of columns be the same, and have the same types, it must also
+ * be the case that the <b>vectors</b> that hold the columns be identical.
+ * Generated code contains references to specific vector objects; passing
+ * along different vectors requires new code to be generated and is treated
+ * as a schema change.
+ * <p>
+ * Drill has no concept of "same schema, different vectors." A change in
+ * vector is just as serious as a change in schema. Hence, operators
+ * try to use the same vectors for their entire lives. That is the change
+ * tracked here.
+ */
+
+// TODO: Does not handle SV4 situations
+
+public class SchemaTracker {
+
+  private int schemaVersion;
+  private BatchSchema currentSchema;
+  private List<ValueVector> currentVectors = new ArrayList<>();
+
+  public void trackSchema(VectorContainer newBatch) {
+
+    if (! isSameSchema(newBatch)) {
+      schemaVersion++;
+      captureSchema(newBatch);
+    }
+  }
+
+  private boolean isSameSchema(VectorContainer newBatch) {
+    if (currentVectors.size() != newBatch.getNumberOfColumns()) {
+      return false;
+    }
+
+    // Compare vectors by identity: not just same type,
+    // must be same instance.
+
+    for (int i = 0; i < currentVectors.size(); i++) {
+      if (currentVectors.get(i) != newBatch.getValueVector(i).getValueVector()) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  private void captureSchema(VectorContainer newBatch) {
+    currentVectors.clear();
+    for (VectorWrapper<?> vw : newBatch) {
+      currentVectors.add(vw.getValueVector());
+    }
+    currentSchema = newBatch.getSchema();
+  }
+
+  public int schemaVersion() { return schemaVersion; }
+  public BatchSchema schema() { return currentSchema; }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/69a5f3a9/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/protocol/VectorContainerAccessor.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..e2d78d7
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/protocol/VectorContainerAccessor.java
@@ -0,0 +1,134 @@
+/*
+ * 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.protocol;
+
+import java.util.Collections;
+import java.util.Iterator;
+
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.TypedFieldId;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.record.WritableBatch;
+import org.apache.drill.exec.record.selection.SelectionVector2;
+import org.apache.drill.exec.record.selection.SelectionVector4;
+
+public class VectorContainerAccessor implements BatchAccessor {
+
+  public static class ContainerAndSv2Accessor extends VectorContainerAccessor {
+
+    private SelectionVector2 sv2;
+
+    public void setSelectionVector(SelectionVector2 sv2) {
+      this.sv2 = sv2;
+    }
+
+    @Override
+    public SelectionVector2 getSelectionVector2() {
+      return sv2;
+    }
+  }
+
+  public static class ContainerAndSv4Accessor extends VectorContainerAccessor {
+
+    private SelectionVector4 sv4;
+
+    @Override
+    public SelectionVector4 getSelectionVector4() {
+      return sv4;
+    }
+  }
+
+  private VectorContainer container;
+  private SchemaTracker schemaTracker = new SchemaTracker();
+
+  /**
+   * Set the vector container. Done initially, and any time the schema of
+   * the container may have changed. May be called with the same container
+   * as the previous call, or a different one. A schema change occurs
+   * unless the vectors are identical across the two containers.
+   *
+   * @param container the container that holds vectors to be sent
+   * downstream
+   */
+
+  public void setContainer(VectorContainer container) {
+    this.container = container;
+    if (container != null) {
+      schemaTracker.trackSchema(container);
+    }
+  }
+
+  @Override
+  public BatchSchema getSchema() {
+    return container == null ? null : container.getSchema();
+  }
+
+  @Override
+  public int schemaVersion() { return schemaTracker.schemaVersion(); }
+
+  @Override
+  public int getRowCount() {
+    return container == null ? 0 : container.getRecordCount();
+  }
+
+  @Override
+  public VectorContainer getOutgoingContainer() { return container; }
+
+  @Override
+  public TypedFieldId getValueVectorId(SchemaPath path) {
+    return container.getValueVectorId(path);
+  }
+
+  @Override
+  public VectorWrapper<?> getValueAccessorById(Class<?> clazz, int... ids) {
+    return container.getValueAccessorById(clazz, ids);
+  }
+
+  @Override
+  public WritableBatch getWritableBatch() {
+    return WritableBatch.get(container);
+  }
+
+  @Override
+  public SelectionVector2 getSelectionVector2() {
+    // Throws an exception by default because containers
+    // do not support selection vectors.
+    return container.getSelectionVector2();
+  }
+
+  @Override
+  public SelectionVector4 getSelectionVector4() {
+    // Throws an exception by default because containers
+    // do not support selection vectors.
+     return container.getSelectionVector4();
+  }
+
+  @Override
+  public Iterator<VectorWrapper<?>> iterator() {
+    if (container == null) {
+      return Collections.emptyIterator();
+    } else {
+      return container.iterator();
+    }
+  }
+
+  @Override
+  public void release() { container.zeroVectors(); }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/69a5f3a9/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/protocol/package-info.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/protocol/package-info.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/protocol/package-info.java
new file mode 100644
index 0000000..11af47c
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/protocol/package-info.java
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+/**
+ * Defines a revised implementation of the Drill RecordBatch protocol. This
+ * version separates concerns into specific classes, and creates as single
+ * "shim" class to implement the iterator protocol, deferring to specific
+ * classes as needed.
+ * <p>
+ * This version is an eventual successor to the original implementation which
+ * used the "kitchen sink" pattern to combine all functionality into s single,
+ * large record batch implementation.
+ */
+
+package org.apache.drill.exec.physical.impl.protocol;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/69a5f3a9/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/protocol/TestOperatorRecordBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/protocol/TestOperatorRecordBatch.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/protocol/TestOperatorRecordBatch.java
new file mode 100644
index 0000000..19946dd
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/protocol/TestOperatorRecordBatch.java
@@ -0,0 +1,747 @@
+/*
+ * 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.protocol;
+
+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 static org.junit.Assert.fail;
+
+import java.util.Iterator;
+
+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.expr.TypeHelper;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.config.Limit;
+import org.apache.drill.exec.physical.impl.protocol.VectorContainerAccessor.ContainerAndSv2Accessor;
+import org.apache.drill.exec.proto.UserBitShared.NamePart;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.RecordBatch.IterOutcome;
+import org.apache.drill.exec.record.TypedFieldId;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.vector.IntVector;
+import org.apache.drill.exec.vector.VarCharVector;
+import org.apache.drill.test.SubOperatorTest;
+import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
+import org.apache.drill.test.rowSet.SchemaBuilder;
+import org.junit.Test;
+
+/**
+ * Test the implementation of the Drill Volcano iterator protocol that
+ * wraps the modular operator implementation.
+ */
+
+public class TestOperatorRecordBatch extends SubOperatorTest {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SubOperatorTest.class);
+
+  /**
+   * Mock operator executor that simply tracks each method call
+   * and provides a light-weight vector container. Returns a
+   * defined number of (batches) with an optional schema change.
+   */
+
+  private class MockOperatorExec implements OperatorExec {
+
+    public boolean bindCalled;
+    public boolean buildSchemaCalled;
+    public int nextCalls = 1;
+    public int nextCount;
+    public int schemaChangeAt = -1;
+    public boolean cancelCalled;
+    public boolean closeCalled;
+    public boolean schemaEOF;
+    private final VectorContainerAccessor batchAccessor;
+
+    public MockOperatorExec() {
+      this(mockBatch());
+    }
+
+    public MockOperatorExec(VectorContainer container) {
+      batchAccessor = new VectorContainerAccessor();
+      batchAccessor.setContainer(container);
+    }
+
+    public MockOperatorExec(VectorContainerAccessor accessor) {
+      batchAccessor = accessor;
+    }
+
+    @Override
+    public void bind(OperatorContext context) { bindCalled = true; }
+
+    @Override
+    public BatchAccessor batchAccessor() {
+      return batchAccessor;
+    }
+
+    @Override
+    public boolean buildSchema() { buildSchemaCalled = true; return ! schemaEOF; }
+
+    @Override
+    public boolean next() {
+      nextCount++;
+      if (nextCount > nextCalls) {
+        return false;
+      }
+      if (nextCount == schemaChangeAt) {
+        BatchSchema newSchema = new SchemaBuilder(batchAccessor.getSchema())
+            .add("b", MinorType.VARCHAR)
+            .build();
+        VectorContainer newContainer = new VectorContainer(fixture.allocator(), newSchema);
+        batchAccessor.setContainer(newContainer);
+      }
+      return true;
+    }
+
+    @Override
+    public void cancel() { cancelCalled = true; }
+
+    @Override
+    public void close() {
+      batchAccessor().getOutgoingContainer().clear();
+      closeCalled = true;
+    }
+  }
+
+  private static VectorContainer mockBatch() {
+    VectorContainer container = new VectorContainer(fixture.allocator(), new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .build());
+    container.buildSchema(SelectionVectorMode.NONE);
+    return container;
+  }
+
+  private OperatorRecordBatch makeOpBatch(MockOperatorExec opExec) {
+    // Dummy operator definition
+    PhysicalOperator popConfig = new Limit(null, 0, 100);
+    return new OperatorRecordBatch(fixture.getFragmentContext(), popConfig, opExec);
+  }
+
+  /**
+   * Simulate a normal run: return some batches, encounter a schema change.
+   */
+
+  @Test
+  public void testNormalLifeCycle() {
+    MockOperatorExec opExec = new MockOperatorExec();
+    opExec.nextCalls = 2;
+    opExec.schemaChangeAt = 2;
+    try (OperatorRecordBatch opBatch = makeOpBatch(opExec)) {
+
+      assertSame(fixture.getFragmentContext(), opBatch.fragmentContext());
+      assertNotNull(opBatch.getContext());
+
+      // First call to next() builds schema
+
+      assertEquals(IterOutcome.OK_NEW_SCHEMA, opBatch.next());
+      assertTrue(opExec.bindCalled);
+      assertTrue(opExec.buildSchemaCalled);
+      assertEquals(0, opExec.nextCount);
+
+      // Second call returns the first batch
+
+      assertEquals(IterOutcome.OK, opBatch.next());
+      assertEquals(1, opExec.nextCount);
+
+      // Third call causes a schema change
+
+      assertEquals(IterOutcome.OK_NEW_SCHEMA, opBatch.next());
+      assertEquals(2, opExec.nextCount);
+
+      // Fourth call reaches EOF
+
+      assertEquals(IterOutcome.NONE, opBatch.next());
+      assertEquals(3, opExec.nextCount);
+
+      // Close
+    } catch (Exception e) {
+      fail();
+    }
+
+    assertTrue(opExec.closeCalled);
+    assertFalse(opExec.cancelCalled);
+  }
+
+  /**
+   * Simulate a truncated life cycle: next() is never called. Not a valid part
+   * of the protocol; but should be ready anyway.
+   */
+
+  @Test
+  public void testTruncatedLifeCycle() {
+    MockOperatorExec opExec = new MockOperatorExec();
+    opExec.schemaEOF = true;
+
+    try (OperatorRecordBatch opBatch = makeOpBatch(opExec)) {
+    } catch (Exception e) {
+      fail();
+    }
+    assertTrue(opExec.bindCalled);
+    assertTrue(opExec.closeCalled);
+  }
+
+  /**
+   * Simulate reaching EOF when trying to create the schema.
+   */
+
+  @Test
+  public void testSchemaEOF() {
+    MockOperatorExec opExec = new MockOperatorExec();
+    opExec.schemaEOF = true;
+
+    try (OperatorRecordBatch opBatch = makeOpBatch(opExec)) {
+      assertEquals(IterOutcome.NONE, opBatch.next());
+      assertTrue(opExec.buildSchemaCalled);
+    } catch (Exception e) {
+      fail();
+    }
+    assertTrue(opExec.closeCalled);
+  }
+
+  /**
+   * Simulate reaching EOF on the first batch. This simulated data source
+   * discovered a schema, but had no data.
+   */
+
+  @Test
+  public void testFirstBatchEOF() {
+    MockOperatorExec opExec = new MockOperatorExec();
+    opExec.nextCalls = 0;
+
+    try (OperatorRecordBatch opBatch = makeOpBatch(opExec)) {
+      assertEquals(IterOutcome.OK_NEW_SCHEMA, opBatch.next());
+      assertTrue(opExec.buildSchemaCalled);
+      assertEquals(IterOutcome.NONE, opBatch.next());
+      assertEquals(1, opExec.nextCount);
+    } catch (Exception e) {
+      fail();
+    }
+    assertTrue(opExec.closeCalled);
+  }
+
+  /**
+   * Simulate the caller failing the operator before getting the schema.
+   */
+
+  @Test
+  public void testFailEarly() {
+    MockOperatorExec opExec = new MockOperatorExec();
+    opExec.nextCalls = 2;
+
+    try (OperatorRecordBatch opBatch = makeOpBatch(opExec)) {
+      opBatch.kill(false);
+      assertFalse(opExec.buildSchemaCalled);
+      assertEquals(0, opExec.nextCount);
+      assertFalse(opExec.cancelCalled);
+    } catch (Exception e) {
+      fail();
+    }
+    assertTrue(opExec.closeCalled);
+  }
+
+  /**
+   * Simulate the caller failing the operator before EOF.
+   */
+
+  @Test
+  public void testFailWhileReading() {
+    MockOperatorExec opExec = new MockOperatorExec();
+    opExec.nextCalls = 2;
+
+    try (OperatorRecordBatch opBatch = makeOpBatch(opExec)) {
+      assertEquals(IterOutcome.OK_NEW_SCHEMA, opBatch.next());
+      assertEquals(IterOutcome.OK, opBatch.next());
+      opBatch.kill(false);
+      assertTrue(opExec.cancelCalled);
+    } catch (Exception e) {
+      fail();
+    }
+    assertTrue(opExec.closeCalled);
+  }
+
+  /**
+   * Simulate the caller failing the operator after EOF but before close.
+   * This is a silly time to fail, but have to handle it anyway.
+   */
+
+  @Test
+  public void testFailBeforeClose() {
+    MockOperatorExec opExec = new MockOperatorExec();
+    opExec.nextCalls = 2;
+
+    try (OperatorRecordBatch opBatch = makeOpBatch(opExec)) {
+      assertEquals(IterOutcome.OK_NEW_SCHEMA, opBatch.next());
+      assertEquals(IterOutcome.OK, opBatch.next());
+      assertEquals(IterOutcome.OK, opBatch.next());
+      assertEquals(IterOutcome.NONE, opBatch.next());
+      opBatch.kill(false);
+
+      // Already hit EOF, so fail won't be passed along.
+
+      assertFalse(opExec.cancelCalled);
+    } catch (Exception e) {
+      fail();
+    }
+    assertTrue(opExec.closeCalled);
+  }
+
+  /**
+   * Simulate the caller failing the operator after close.
+   * This is violates the operator protocol, but have to handle it anyway.
+   */
+
+  @Test
+  public void testFailAfterClose() {
+    MockOperatorExec opExec = new MockOperatorExec();
+    opExec.nextCalls = 2;
+
+    OperatorRecordBatch opBatch = makeOpBatch(opExec);
+    assertEquals(IterOutcome.OK_NEW_SCHEMA, opBatch.next());
+    assertEquals(IterOutcome.OK, opBatch.next());
+    assertEquals(IterOutcome.OK, opBatch.next());
+    assertEquals(IterOutcome.NONE, opBatch.next());
+    try {
+      opBatch.close();
+    } catch (Exception e) {
+      fail();
+    }
+    assertTrue(opExec.closeCalled);
+    opBatch.kill(false);
+    assertFalse(opExec.cancelCalled);
+  }
+
+  /**
+   * The record batch abstraction has a bunch of methods to work with a vector container.
+   * Rather than simply exposing the container itself, the batch instead exposes various
+   * container operations. Probably an artifact of its history. In any event, make
+   * sure those methods are passed through to the container accessor.
+   */
+
+  @Test
+  public void testBatchAccessor() {
+    BatchSchema schema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .add("b", MinorType.VARCHAR)
+        .build();
+    SingleRowSet rs = fixture.rowSetBuilder(schema)
+        .addRow(10, "fred")
+        .addRow(20, "wilma")
+        .build();
+    MockOperatorExec opExec = new MockOperatorExec(rs.container());
+    opExec.nextCalls = 1;
+
+    try (OperatorRecordBatch opBatch = makeOpBatch(opExec)) {
+      assertEquals(IterOutcome.OK_NEW_SCHEMA, opBatch.next());
+      assertEquals(schema, opBatch.getSchema());
+      assertEquals(2, opBatch.getRecordCount());
+      assertSame(rs.container(), opBatch.getOutgoingContainer());
+
+      Iterator<VectorWrapper<?>> iter = opBatch.iterator();
+      assertEquals("a", iter.next().getValueVector().getField().getName());
+      assertEquals("b", iter.next().getValueVector().getField().getName());
+
+      // Not a full test of the schema path; just make sure that the
+      // pass-through to the Vector Container works.
+
+      SchemaPath path = SchemaPath.create(NamePart.newBuilder().setName("a").build());
+      TypedFieldId id = opBatch.getValueVectorId(path);
+      assertEquals(MinorType.INT, id.getFinalType().getMinorType());
+      assertEquals(1, id.getFieldIds().length);
+      assertEquals(0, id.getFieldIds()[0]);
+
+      path = SchemaPath.create(NamePart.newBuilder().setName("b").build());
+      id = opBatch.getValueVectorId(path);
+      assertEquals(MinorType.VARCHAR, id.getFinalType().getMinorType());
+      assertEquals(1, id.getFieldIds().length);
+      assertEquals(1, id.getFieldIds()[0]);
+
+      // Sanity check of getValueAccessorById()
+
+      VectorWrapper<?> w = opBatch.getValueAccessorById(IntVector.class, 0);
+      assertNotNull(w);
+      assertEquals("a", w.getValueVector().getField().getName());
+      w = opBatch.getValueAccessorById(VarCharVector.class, 1);
+      assertNotNull(w);
+      assertEquals("b", w.getValueVector().getField().getName());
+
+      // getWritableBatch() ?
+
+      // No selection vectors
+
+      try {
+        opBatch.getSelectionVector2();
+        fail();
+      } catch (UnsupportedOperationException e) {
+        // Expected
+      }
+      try {
+        opBatch.getSelectionVector4();
+        fail();
+      } catch (UnsupportedOperationException e) {
+        // Expected
+      }
+
+    } catch (Exception e) {
+      fail(e.getMessage());
+    }
+    assertTrue(opExec.closeCalled);
+  }
+
+  @Test
+  public void testSchemaChange() {
+    BatchSchema schema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .add("b", MinorType.VARCHAR)
+        .build();
+    SingleRowSet rs = fixture.rowSetBuilder(schema)
+        .addRow(10, "fred")
+        .addRow(20, "wilma")
+        .build();
+    VectorContainer container = rs.container();
+    MockOperatorExec opExec = new MockOperatorExec(container);
+    int schemaVersion = opExec.batchAccessor().schemaVersion();
+
+    // Be tidy: start at 1.
+
+    assertEquals(1, schemaVersion);
+
+    // Changing data does not trigger schema change
+
+    container.zeroVectors();
+    opExec.batchAccessor.setContainer(container);
+    assertEquals(schemaVersion, opExec.batchAccessor().schemaVersion());
+
+    // Different container, same vectors, does not trigger a change
+
+    VectorContainer c2 = new VectorContainer(fixture.allocator());
+    for (VectorWrapper<?> vw : container) {
+      c2.add(vw.getValueVector());
+    }
+    c2.buildSchema(SelectionVectorMode.NONE);
+    opExec.batchAccessor.setContainer(c2);
+    assertEquals(schemaVersion, opExec.batchAccessor().schemaVersion());
+
+    opExec.batchAccessor.setContainer(container);
+    assertEquals(schemaVersion, opExec.batchAccessor().schemaVersion());
+
+    // Replacing a vector with another of the same type does trigger
+    // a change.
+
+    VectorContainer c3 = new VectorContainer(fixture.allocator());
+    c3.add(container.getValueVector(0).getValueVector());
+    c3.add(TypeHelper.getNewVector(
+            container.getValueVector(1).getValueVector().getField(),
+            fixture.allocator(), null));
+    c3.buildSchema(SelectionVectorMode.NONE);
+    opExec.batchAccessor.setContainer(c3);
+    assertEquals(schemaVersion + 1, opExec.batchAccessor().schemaVersion());
+    schemaVersion = opExec.batchAccessor().schemaVersion();
+
+    // No change if same schema again
+
+    opExec.batchAccessor.setContainer(c3);
+    assertEquals(schemaVersion, opExec.batchAccessor().schemaVersion());
+
+    // Adding a vector triggers a change
+
+    MaterializedField c = SchemaBuilder.columnSchema("c", MinorType.INT, DataMode.OPTIONAL);
+    c3.add(TypeHelper.getNewVector(c, fixture.allocator(), null));
+    c3.buildSchema(SelectionVectorMode.NONE);
+    opExec.batchAccessor.setContainer(c3);
+    assertEquals(schemaVersion + 1, opExec.batchAccessor().schemaVersion());
+    schemaVersion = opExec.batchAccessor().schemaVersion();
+
+    // No change if same schema again
+
+    opExec.batchAccessor.setContainer(c3);
+    assertEquals(schemaVersion, opExec.batchAccessor().schemaVersion());
+
+    // Removing a vector triggers a change
+
+    c3.remove(c3.getValueVector(2).getValueVector());
+    c3.buildSchema(SelectionVectorMode.NONE);
+    assertEquals(2, c3.getNumberOfColumns());
+    opExec.batchAccessor.setContainer(c3);
+    assertEquals(schemaVersion + 1, opExec.batchAccessor().schemaVersion());
+    schemaVersion = opExec.batchAccessor().schemaVersion();
+
+    // Clean up
+
+    opExec.close();
+    c2.clear();
+    c3.clear();
+  }
+
+  /**
+   * Test that an SV2 is properly handled by the proper container accessor.
+   */
+
+  @Test
+  public void testSv2() {
+    BatchSchema schema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .add("b", MinorType.VARCHAR)
+        .build();
+    SingleRowSet rs = fixture.rowSetBuilder(schema)
+        .addRow(10, "fred")
+        .addRow(20, "wilma")
+        .withSv2()
+        .build();
+
+    ContainerAndSv2Accessor accessor = new ContainerAndSv2Accessor();
+    accessor.setContainer(rs.container());
+    accessor.setSelectionVector(rs.getSv2());
+
+    MockOperatorExec opExec = new MockOperatorExec(accessor);
+    opExec.nextCalls = 1;
+
+    try (OperatorRecordBatch opBatch = makeOpBatch(opExec)) {
+      assertEquals(IterOutcome.OK_NEW_SCHEMA, opBatch.next());
+      assertSame(rs.getSv2(), opBatch.getSelectionVector2());
+
+    } catch (Exception e) {
+      fail();
+    }
+    assertTrue(opExec.closeCalled);
+
+    // Must release SV2
+
+    rs.clear();
+  }
+
+  //-----------------------------------------------------------------------
+  // Exception error cases
+  //
+  // Assumes that any of the operator executor methods could throw an
+  // exception. A wise implementation will throw a user exception that the
+  // operator just passes along. A lazy implementation will throw any old
+  // unchecked exception. Validate both cases.
+
+  public static final String ERROR_MSG = "My Bad!";
+
+  /**
+   * Failure on the bind method.
+   */
+
+  @Test
+  public void testWrappedExceptionOnBind() {
+    MockOperatorExec opExec = new MockOperatorExec() {
+      @Override
+      public void bind(OperatorContext context) {
+         throw new IllegalStateException(ERROR_MSG);
+      }
+    };
+    try (OperatorRecordBatch opBatch = makeOpBatch(opExec)) {
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains(ERROR_MSG));
+      assertTrue(e.getCause() instanceof IllegalStateException);
+    } catch (Throwable t) {
+      fail();
+    }
+    assertFalse(opExec.cancelCalled); // Cancel not called: too early in life
+    assertFalse(opExec.closeCalled); // Same with close
+  }
+
+  @Test
+  public void testUserExceptionOnBind() {
+    MockOperatorExec opExec = new MockOperatorExec() {
+      @Override
+      public void bind(OperatorContext context) {
+         throw UserException.connectionError()
+           .message(ERROR_MSG)
+           .build(logger);
+      }
+    };
+    try (OperatorRecordBatch opBatch = makeOpBatch(opExec)) {
+      opBatch.next();
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains(ERROR_MSG));
+      assertNull(e.getCause());
+    } catch (Throwable t) {
+      fail();
+    }
+    assertFalse(opExec.cancelCalled); // Cancel not called: too early in life
+    assertFalse(opExec.closeCalled); // Same with close
+  }
+
+  /**
+   * Failure when building the schema (first call to next()).
+   */
+  @Test
+  public void testWrappedExceptionOnBuildSchema() {
+    MockOperatorExec opExec = new MockOperatorExec() {
+      @Override
+      public boolean buildSchema() {
+         throw new IllegalStateException(ERROR_MSG);
+      }
+    };
+    try (OperatorRecordBatch opBatch = makeOpBatch(opExec)) {
+      opBatch.next();
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains(ERROR_MSG));
+      assertTrue(e.getCause() instanceof IllegalStateException);
+    } catch (Throwable t) {
+      fail();
+    }
+    assertTrue(opExec.cancelCalled);
+    assertTrue(opExec.closeCalled);
+  }
+
+  @Test
+  public void testUserExceptionOnBuildSchema() {
+    MockOperatorExec opExec = new MockOperatorExec() {
+      @Override
+      public boolean buildSchema() {
+        throw UserException.dataReadError()
+            .message(ERROR_MSG)
+            .build(logger);
+      }
+    };
+    try (OperatorRecordBatch opBatch = makeOpBatch(opExec)) {
+      opBatch.next();
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains(ERROR_MSG));
+      assertNull(e.getCause());
+    } catch (Throwable t) {
+      fail();
+    }
+    assertTrue(opExec.cancelCalled);
+    assertTrue(opExec.closeCalled);
+  }
+
+  /**
+   * Failure on the second or subsequent calls to next(), when actually
+   * fetching a record batch.
+   */
+
+  @Test
+  public void testWrappedExceptionOnNext() {
+    MockOperatorExec opExec = new MockOperatorExec() {
+      @Override
+      public boolean next() {
+         throw new IllegalStateException(ERROR_MSG);
+      }
+    };
+    try (OperatorRecordBatch opBatch = makeOpBatch(opExec)) {
+      assertEquals(IterOutcome.OK_NEW_SCHEMA, opBatch.next());
+      opBatch.next();
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains(ERROR_MSG));
+      assertTrue(e.getCause() instanceof IllegalStateException);
+    } catch (Throwable t) {
+      fail();
+    }
+    assertTrue(opExec.cancelCalled);
+    assertTrue(opExec.closeCalled);
+  }
+
+  @Test
+  public void testUserExceptionOnNext() {
+    MockOperatorExec opExec = new MockOperatorExec() {
+      @Override
+      public boolean next() {
+        throw UserException.dataReadError()
+              .message(ERROR_MSG)
+              .build(logger);
+      }
+    };
+    try (OperatorRecordBatch opBatch = makeOpBatch(opExec)) {
+      assertEquals(IterOutcome.OK_NEW_SCHEMA, opBatch.next());
+      opBatch.next();
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains(ERROR_MSG));
+      assertNull(e.getCause());
+    } catch (Throwable t) {
+      fail();
+    }
+    assertTrue(opExec.cancelCalled);
+    assertTrue(opExec.closeCalled);
+  }
+
+  /**
+   * Failure when closing the operator implementation.
+   */
+
+  @Test
+  public void testWrappedExceptionOnClose() {
+    MockOperatorExec opExec = new MockOperatorExec() {
+      @Override
+      public void close() {
+        // Release memory
+        super.close();
+        // Then fail
+        throw new IllegalStateException(ERROR_MSG);
+      }
+    };
+    opExec.nextCalls = 1;
+    try (OperatorRecordBatch opBatch = makeOpBatch(opExec)) {
+      assertEquals(IterOutcome.OK_NEW_SCHEMA, opBatch.next());
+      assertEquals(IterOutcome.OK, opBatch.next());
+      assertEquals(IterOutcome.NONE, opBatch.next());
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains(ERROR_MSG));
+      assertTrue(e.getCause() instanceof IllegalStateException);
+    } catch (Throwable t) {
+      fail();
+    }
+    assertFalse(opExec.cancelCalled);
+    assertTrue(opExec.closeCalled);
+  }
+
+  @Test
+  public void testUserExceptionOnClose() {
+    MockOperatorExec opExec = new MockOperatorExec() {
+      @Override
+      public void close() {
+        // Release memory
+        super.close();
+        // Then fail
+        throw UserException.dataReadError()
+              .message(ERROR_MSG)
+              .build(logger);
+      }
+    };
+    try (OperatorRecordBatch opBatch = makeOpBatch(opExec)) {
+      assertEquals(IterOutcome.OK_NEW_SCHEMA, opBatch.next());
+      assertEquals(IterOutcome.OK, opBatch.next());
+      assertEquals(IterOutcome.NONE, opBatch.next());
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains(ERROR_MSG));
+      assertNull(e.getCause());
+    } catch (Throwable t) {
+      fail();
+    }
+    assertFalse(opExec.cancelCalled);
+    assertTrue(opExec.closeCalled);
+  }
+}


[13/17] drill git commit: DRILL-6180: Use System Option "output_batch_size" for External Sort

Posted by ar...@apache.org.
DRILL-6180: Use System Option "output_batch_size" for External Sort

closes #1129


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/4ee207bd
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/4ee207bd
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/4ee207bd

Branch: refs/heads/master
Commit: 4ee207bd6744fcd9efcd7bafa3559093ad89fb46
Parents: 6a55b2b
Author: Padma Penumarthy <pp...@yahoo.com>
Authored: Thu Feb 22 16:41:47 2018 -0800
Committer: Arina Ielchiieva <ar...@gmail.com>
Committed: Sat Mar 3 19:47:51 2018 +0200

----------------------------------------------------------------------
 .../impl/xsort/managed/ExternalSortBatch.java   |  2 +-
 .../physical/impl/xsort/managed/SortConfig.java | 12 ++-
 .../src/main/resources/drill-module.conf        |  9 +-
 .../managed/TestExternalSortInternals.java      | 88 ++++++++++----------
 .../impl/xsort/managed/TestSortImpl.java        |  3 +-
 5 files changed, 59 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/4ee207bd/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/ExternalSortBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/ExternalSortBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/ExternalSortBatch.java
index 23e66a0..ea53a80 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/ExternalSortBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/ExternalSortBatch.java
@@ -213,7 +213,7 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
     super(popConfig, context, true);
     this.incoming = incoming;
 
-    SortConfig sortConfig = new SortConfig(context.getConfig());
+    SortConfig sortConfig = new SortConfig(context.getConfig(), context.getOptions());
     SpillSet spillSet = new SpillSet(context.getConfig(), context.getHandle(), popConfig);
     oContext.setInjector(injector);
     PriorityQueueCopierWrapper copierHolder = new PriorityQueueCopierWrapper(oContext);

http://git-wip-us.apache.org/repos/asf/drill/blob/4ee207bd/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortConfig.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortConfig.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortConfig.java
index 236c2f3..e592ccb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortConfig.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortConfig.java
@@ -19,6 +19,7 @@ package org.apache.drill.exec.physical.impl.xsort.managed;
 
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.server.options.OptionManager;
 
 public class SortConfig {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExternalSortBatch.class);
@@ -71,8 +72,7 @@ public class SortConfig {
 
   private final int mSortBatchSize;
 
-  public SortConfig(DrillConfig config) {
-
+  public SortConfig(DrillConfig config, OptionManager options) {
     // Optional configured memory limit, typically used only for testing.
 
     maxMemory = config.getBytes(ExecConstants.EXTERNAL_SORT_MAX_MEMORY);
@@ -99,7 +99,13 @@ public class SortConfig {
     // of memory, but no smaller than the minimum size. In any event, an
     // output batch can contain no fewer than a single record.
 
-    mergeBatchSize = (int) Math.max(config.getBytes(ExecConstants.EXTERNAL_SORT_MERGE_BATCH_SIZE), MIN_MERGE_BATCH_SIZE);
+    // get the output batch size from context.
+    // Size of the batch sent downstream from the sort operator during
+    // the merge phase. Default value is 16M.
+    // Don't change defaults unless you know what you are doing,
+    // larger sizes can result in memory fragmentation, smaller sizes
+    // in excessive operator iterator overhead.
+    mergeBatchSize = (int) Math.max(options.getOption(ExecConstants.OUTPUT_BATCH_SIZE_VALIDATOR), MIN_MERGE_BATCH_SIZE);
 
     // Limit on in-memory batches, primarily for testing.
 

http://git-wip-us.apache.org/repos/asf/drill/blob/4ee207bd/exec/java-exec/src/main/resources/drill-module.conf
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/drill-module.conf b/exec/java-exec/src/main/resources/drill-module.conf
index fc365d2..32be387 100644
--- a/exec/java-exec/src/main/resources/drill-module.conf
+++ b/exec/java-exec/src/main/resources/drill-module.conf
@@ -293,12 +293,7 @@ drill.exec: {
         // Set large enough to get long, continuous writes, but not so
         // large as to overwhelm a temp directory.
         // Supports HOCON memory suffixes.
-        file_size: 256M,
-        // Size of the batch sent downstream from the sort operator during
-        // the merge phase. Don't change this unless you know what you are doing,
-        // larger sizes can result in memory fragmentation, smaller sizes
-        // in excessive operator iterator overhead.
-        merge_batch_size = 16M
+        file_size: 256M
       }
     }
   },
@@ -424,7 +419,7 @@ drill.exec.options: {
     drill.exec.storage.implicit.fqn.column.label: "fqn",
     drill.exec.storage.implicit.suffix.column.label: "suffix",
     drill.exec.testing.controls: "{}",
-    drill.exec.memory.operator.output_batch_size : 33554432, # 32 MB
+    drill.exec.memory.operator.output_batch_size : 16777216, # 16 MB
     exec.bulk_load_table_list.bulk_size: 1000,
     exec.compile.scalar_replacement: false,
     exec.enable_bulk_load_table_list: false,

http://git-wip-us.apache.org/repos/asf/drill/blob/4ee207bd/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestExternalSortInternals.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestExternalSortInternals.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestExternalSortInternals.java
index 1315a86..b0afbb2 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestExternalSortInternals.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestExternalSortInternals.java
@@ -22,12 +22,13 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 import org.apache.drill.categories.OperatorTest;
-import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.ops.OperatorStats;
 import org.apache.drill.exec.physical.impl.xsort.managed.SortMemoryManager.MergeAction;
 import org.apache.drill.exec.physical.impl.xsort.managed.SortMemoryManager.MergeTask;
-import org.apache.drill.test.ConfigBuilder;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.test.OperatorFixture;
 import org.apache.drill.test.SubOperatorTest;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -42,8 +43,7 @@ public class TestExternalSortInternals extends SubOperatorTest {
    */
   @Test
   public void testConfigDefaults() {
-    DrillConfig drillConfig = DrillConfig.create();
-    SortConfig sortConfig = new SortConfig(drillConfig);
+    SortConfig sortConfig = new SortConfig(fixture.getFragmentContext().getConfig(), fixture.getOptionManager());
     // Zero means no artificial limit
     assertEquals(0, sortConfig.maxMemory());
     // Zero mapped to large number
@@ -67,16 +67,18 @@ public class TestExternalSortInternals extends SubOperatorTest {
   @Test
   public void testConfigOverride() {
     // Verify the various HOCON ways of setting memory
-    DrillConfig drillConfig = new ConfigBuilder()
+    OperatorFixture.Builder builder = new OperatorFixture.Builder();
+    builder.configBuilder()
         .put(ExecConstants.EXTERNAL_SORT_MAX_MEMORY, "2000K")
         .put(ExecConstants.EXTERNAL_SORT_MERGE_LIMIT, 10)
         .put(ExecConstants.EXTERNAL_SORT_SPILL_FILE_SIZE, "10M")
         .put(ExecConstants.EXTERNAL_SORT_SPILL_BATCH_SIZE, 500_000)
-        .put(ExecConstants.EXTERNAL_SORT_MERGE_BATCH_SIZE, 600_000)
         .put(ExecConstants.EXTERNAL_SORT_BATCH_LIMIT, 50)
         .put(ExecConstants.EXTERNAL_SORT_MSORT_MAX_BATCHSIZE, 10)
         .build();
-    SortConfig sortConfig = new SortConfig(drillConfig);
+    FragmentContext fragmentContext = builder.build().getFragmentContext();
+    fragmentContext.getOptions().setLocalOption(ExecConstants.OUTPUT_BATCH_SIZE, 600_000);
+    SortConfig sortConfig = new SortConfig(fragmentContext.getConfig(), fragmentContext.getOptions());
     assertEquals(2000 * 1024, sortConfig.maxMemory());
     assertEquals(10, sortConfig.mergeLimit());
     assertEquals(10 * ONE_MEG, sortConfig.spillFileSize());
@@ -91,15 +93,17 @@ public class TestExternalSortInternals extends SubOperatorTest {
    */
   @Test
   public void testConfigLimits() {
-    DrillConfig drillConfig = new ConfigBuilder()
+    OperatorFixture.Builder builder = new OperatorFixture.Builder();
+    builder.configBuilder()
         .put(ExecConstants.EXTERNAL_SORT_MERGE_LIMIT, SortConfig.MIN_MERGE_LIMIT - 1)
         .put(ExecConstants.EXTERNAL_SORT_SPILL_FILE_SIZE, SortConfig.MIN_SPILL_FILE_SIZE - 1)
         .put(ExecConstants.EXTERNAL_SORT_SPILL_BATCH_SIZE, SortConfig.MIN_SPILL_BATCH_SIZE - 1)
-        .put(ExecConstants.EXTERNAL_SORT_MERGE_BATCH_SIZE, SortConfig.MIN_MERGE_BATCH_SIZE - 1)
         .put(ExecConstants.EXTERNAL_SORT_BATCH_LIMIT, 1)
         .put(ExecConstants.EXTERNAL_SORT_MSORT_MAX_BATCHSIZE, 0)
         .build();
-    SortConfig sortConfig = new SortConfig(drillConfig);
+    FragmentContext fragmentContext = builder.build().getFragmentContext();
+    fragmentContext.getOptions().setLocalOption(ExecConstants.OUTPUT_BATCH_SIZE, SortConfig.MIN_MERGE_BATCH_SIZE - 1);
+    SortConfig sortConfig = new SortConfig(fragmentContext.getConfig(), fragmentContext.getOptions());
     assertEquals(SortConfig.MIN_MERGE_LIMIT, sortConfig.mergeLimit());
     assertEquals(SortConfig.MIN_SPILL_FILE_SIZE, sortConfig.spillFileSize());
     assertEquals(SortConfig.MIN_SPILL_BATCH_SIZE, sortConfig.spillBatchSize());
@@ -110,8 +114,7 @@ public class TestExternalSortInternals extends SubOperatorTest {
 
   @Test
   public void testMemoryManagerBasics() {
-    DrillConfig drillConfig = DrillConfig.create();
-    SortConfig sortConfig = new SortConfig(drillConfig);
+    SortConfig sortConfig = new SortConfig(fixture.getFragmentContext().getConfig(), fixture.getFragmentContext().getOptions());
     long memoryLimit = 70 * ONE_MEG;
     SortMemoryManager memManager = new SortMemoryManager(sortConfig, memoryLimit);
 
@@ -217,8 +220,7 @@ public class TestExternalSortInternals extends SubOperatorTest {
 
   @Test
   public void testSmallRows() {
-    DrillConfig drillConfig = DrillConfig.create();
-    SortConfig sortConfig = new SortConfig(drillConfig);
+    SortConfig sortConfig = new SortConfig(fixture.getFragmentContext().getConfig(), fixture.getFragmentContext().getOptions());
     long memoryLimit = 100 * ONE_MEG;
     SortMemoryManager memManager = new SortMemoryManager(sortConfig, memoryLimit);
 
@@ -263,8 +265,7 @@ public class TestExternalSortInternals extends SubOperatorTest {
 
   @Test
   public void testLowMemory() {
-    DrillConfig drillConfig = DrillConfig.create();
-    SortConfig sortConfig = new SortConfig(drillConfig);
+    SortConfig sortConfig = new SortConfig(fixture.getFragmentContext().getConfig(), fixture.getFragmentContext().getOptions());
     int memoryLimit = 10 * ONE_MEG;
     SortMemoryManager memManager = new SortMemoryManager(sortConfig, memoryLimit);
 
@@ -306,8 +307,7 @@ public class TestExternalSortInternals extends SubOperatorTest {
 
   @Test
   public void testLowerMemory() {
-    DrillConfig drillConfig = DrillConfig.create();
-    SortConfig sortConfig = new SortConfig(drillConfig);
+    SortConfig sortConfig = new SortConfig(fixture.getFragmentContext().getConfig(), fixture.getFragmentContext().getOptions());
     int memoryLimit = 10 * ONE_MEG;
     SortMemoryManager memManager = new SortMemoryManager(sortConfig, memoryLimit);
 
@@ -352,8 +352,7 @@ public class TestExternalSortInternals extends SubOperatorTest {
 
   @Test
   public void testExtremeLowMemory() {
-    DrillConfig drillConfig = DrillConfig.create();
-    SortConfig sortConfig = new SortConfig(drillConfig);
+    SortConfig sortConfig = new SortConfig(fixture.getFragmentContext().getConfig(), fixture.getFragmentContext().getOptions());
     long memoryLimit = 10 * ONE_MEG;
     SortMemoryManager memManager = new SortMemoryManager(sortConfig, memoryLimit);
 
@@ -391,8 +390,7 @@ public class TestExternalSortInternals extends SubOperatorTest {
 
   @Test
   public void testMemoryOverflow() {
-    DrillConfig drillConfig = DrillConfig.create();
-    SortConfig sortConfig = new SortConfig(drillConfig);
+    SortConfig sortConfig = new SortConfig(fixture.getFragmentContext().getConfig(), fixture.getFragmentContext().getOptions());
     long memoryLimit = 10 * ONE_MEG;
     SortMemoryManager memManager = new SortMemoryManager(sortConfig, memoryLimit);
 
@@ -413,34 +411,36 @@ public class TestExternalSortInternals extends SubOperatorTest {
 
   @Test
   public void testConfigConstraints() {
-    int memConstaint = 40 * ONE_MEG;
-    int batchSizeConstaint = ONE_MEG / 2;
-    int mergeSizeConstaint = ONE_MEG;
-    DrillConfig drillConfig = new ConfigBuilder()
-        .put(ExecConstants.EXTERNAL_SORT_MAX_MEMORY, memConstaint)
-        .put(ExecConstants.EXTERNAL_SORT_SPILL_BATCH_SIZE, batchSizeConstaint)
-        .put(ExecConstants.EXTERNAL_SORT_MERGE_BATCH_SIZE, mergeSizeConstaint)
+    int memConstraint = 40 * ONE_MEG;
+    int batchSizeConstraint = ONE_MEG / 2;
+    int mergeSizeConstraint = ONE_MEG;
+
+    OperatorFixture.Builder builder = new OperatorFixture.Builder();
+    builder.configBuilder()
+        .put(ExecConstants.EXTERNAL_SORT_MAX_MEMORY, memConstraint)
+        .put(ExecConstants.EXTERNAL_SORT_SPILL_BATCH_SIZE, batchSizeConstraint)
         .build();
-    SortConfig sortConfig = new SortConfig(drillConfig);
+    FragmentContext fragmentContext = builder.build().getFragmentContext();
+    fragmentContext.getOptions().setLocalOption(ExecConstants.OUTPUT_BATCH_SIZE, mergeSizeConstraint);
+    SortConfig sortConfig = new SortConfig(fragmentContext.getConfig(), fragmentContext.getOptions());
     long memoryLimit = 50 * ONE_MEG;
     SortMemoryManager memManager = new SortMemoryManager(sortConfig, memoryLimit);
 
-    assertEquals(batchSizeConstaint, memManager.getPreferredSpillBatchSize());
-    assertEquals(mergeSizeConstaint, memManager.getPreferredMergeBatchSize());
-    assertEquals(memConstaint, memManager.getMemoryLimit());
+    assertEquals(batchSizeConstraint, memManager.getPreferredSpillBatchSize());
+    assertEquals(mergeSizeConstraint, memManager.getPreferredMergeBatchSize());
+    assertEquals(memConstraint, memManager.getMemoryLimit());
 
     int rowWidth = 300;
     int rowCount = 10000;
     int batchSize = rowWidth * rowCount * 2;
 
     memManager.updateEstimates(batchSize, rowWidth, rowCount);
-    verifyCalcs(sortConfig, memConstaint, memManager, batchSize, rowWidth, rowCount);
+    verifyCalcs(sortConfig, memConstraint, memManager, batchSize, rowWidth, rowCount);
   }
 
   @Test
   public void testMemoryDynamics() {
-    DrillConfig drillConfig = DrillConfig.create();
-    SortConfig sortConfig = new SortConfig(drillConfig);
+    SortConfig sortConfig = new SortConfig(fixture.getFragmentContext().getConfig(), fixture.getFragmentContext().getOptions());
     long memoryLimit = 50 * ONE_MEG;
     SortMemoryManager memManager = new SortMemoryManager(sortConfig, memoryLimit);
 
@@ -471,10 +471,12 @@ public class TestExternalSortInternals extends SubOperatorTest {
     // No artificial merge limit
 
     int mergeLimitConstraint = 100;
-    DrillConfig drillConfig = new ConfigBuilder()
+    OperatorFixture.Builder builder = new OperatorFixture.Builder();
+    builder.configBuilder()
         .put(ExecConstants.EXTERNAL_SORT_MERGE_LIMIT, mergeLimitConstraint)
         .build();
-    SortConfig sortConfig = new SortConfig(drillConfig);
+    FragmentContext fragmentContext = builder.build().getFragmentContext();
+    SortConfig sortConfig = new SortConfig(fragmentContext.getConfig(), fragmentContext.getOptions());
 
     // Allow four spill batches, 8 MB each, plus one output of 16
     // Allow for internal fragmentation
@@ -569,9 +571,7 @@ public class TestExternalSortInternals extends SubOperatorTest {
 
   @Test
   public void testMergeCalcsExtreme() {
-
-    DrillConfig drillConfig = DrillConfig.create();
-    SortConfig sortConfig = new SortConfig(drillConfig);
+    SortConfig sortConfig = new SortConfig(fixture.getFragmentContext().getConfig(), fixture.getFragmentContext().getOptions());
 
     // Force odd situation in which the spill batch is larger
     // than memory. Won't actually run, but needed to test
@@ -600,10 +600,12 @@ public class TestExternalSortInternals extends SubOperatorTest {
   public void testMergeLimit() {
     // Constrain merge width
     int mergeLimitConstraint = 5;
-    DrillConfig drillConfig = new ConfigBuilder()
+    OperatorFixture.Builder builder = new OperatorFixture.Builder();
+    builder.configBuilder()
         .put(ExecConstants.EXTERNAL_SORT_MERGE_LIMIT, mergeLimitConstraint)
         .build();
-    SortConfig sortConfig = new SortConfig(drillConfig);
+    FragmentContext fragmentContext = builder.build().getFragmentContext();
+    SortConfig sortConfig = new SortConfig(fragmentContext.getConfig(), fragmentContext.getOptions());
     // Plenty of memory, memory will not be a limit
     long memoryLimit = 400 * ONE_MEG;
     SortMemoryManager memManager = new SortMemoryManager(sortConfig, memoryLimit);

http://git-wip-us.apache.org/repos/asf/drill/blob/4ee207bd/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSortImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSortImpl.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSortImpl.java
index a985478..bcc53fa 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSortImpl.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSortImpl.java
@@ -30,6 +30,7 @@ import org.apache.drill.common.logical.data.Order.Ordering;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.ops.OperatorContext;
 import org.apache.drill.exec.physical.config.Sort;
 import org.apache.drill.exec.physical.impl.spill.SpillSet;
@@ -94,7 +95,7 @@ public class TestSortImpl extends DrillTest {
           .setMinorFragmentId(3)
           .setQueryId(queryId)
           .build();
-    SortConfig sortConfig = new SortConfig(opContext.getFragmentContext().getConfig());
+    SortConfig sortConfig = new SortConfig(opContext.getFragmentContext().getConfig(), opContext.getFragmentContext().getOptions());
 
     SpillSet spillSet = new SpillSet(opContext.getFragmentContext().getConfig(), handle, popConfig);
     PriorityQueueCopierWrapper copierHolder = new PriorityQueueCopierWrapper(opContext);


[04/17] drill git commit: DRILL-6204: Pass tables columns without partition columns to empty Hive reader

Posted by ar...@apache.org.
DRILL-6204: Pass tables columns without partition columns to empty Hive reader

closes #1146


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/1697e531
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/1697e531
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/1697e531

Branch: refs/heads/master
Commit: 1697e531dce5fd094f3b5d4a29448df704ed6c4a
Parents: 6bf04ec
Author: Arina Ielchiieva <ar...@gmail.com>
Authored: Fri Mar 2 13:38:00 2018 +0200
Committer: Arina Ielchiieva <ar...@gmail.com>
Committed: Sat Mar 3 19:47:36 2018 +0200

----------------------------------------------------------------------
 .../exec/store/hive/HiveDrillNativeScanBatchCreator.java  | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/1697e531/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeScanBatchCreator.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeScanBatchCreator.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeScanBatchCreator.java
index 3861aa0..43318d1 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeScanBatchCreator.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeScanBatchCreator.java
@@ -76,11 +76,11 @@ public class HiveDrillNativeScanBatchCreator implements BatchCreator<HiveDrillNa
 
     final List<String[]> partitionColumns = Lists.newArrayList();
     final List<Integer> selectedPartitionColumns = Lists.newArrayList();
-    List<SchemaPath> newColumns = columns;
+    List<SchemaPath> tableColumns = columns;
     if (!selectAllQuery) {
       // Separate out the partition and non-partition columns. Non-partition columns are passed directly to the
       // ParquetRecordReader. Partition columns are passed to ScanBatch.
-      newColumns = Lists.newArrayList();
+      tableColumns = Lists.newArrayList();
       Pattern pattern = Pattern.compile(String.format("%s[0-9]+", partitionDesignator));
       for (SchemaPath column : columns) {
         Matcher m = pattern.matcher(column.getRootSegmentPath());
@@ -88,7 +88,7 @@ public class HiveDrillNativeScanBatchCreator implements BatchCreator<HiveDrillNa
           selectedPartitionColumns.add(
               Integer.parseInt(column.getRootSegmentPath().substring(partitionDesignator.length())));
         } else {
-          newColumns.add(column);
+          tableColumns.add(column);
         }
       }
     }
@@ -139,7 +139,7 @@ public class HiveDrillNativeScanBatchCreator implements BatchCreator<HiveDrillNa
                 CodecFactory.createDirectCodecFactory(fs.getConf(),
                     new ParquetDirectByteBufferAllocator(oContext.getAllocator()), 0),
                 parquetMetadata,
-                newColumns,
+                tableColumns,
                 containsCorruptDates)
             );
             Map<String, String> implicitValues = Maps.newLinkedHashMap();
@@ -174,7 +174,7 @@ public class HiveDrillNativeScanBatchCreator implements BatchCreator<HiveDrillNa
     // If there are no readers created (which is possible when the table is empty or no row groups are matched),
     // create an empty RecordReader to output the schema
     if (readers.size() == 0) {
-      readers.add(new HiveDefaultReader(table, null, null, columns, context, conf,
+      readers.add(new HiveDefaultReader(table, null, null, tableColumns, context, conf,
         ImpersonationUtil.createProxyUgi(config.getUserName(), context.getQueryUserName())));
     }
 


[12/17] drill git commit: DRILL-6099: Push limit past flatten(project) without pushdown into scan

Posted by ar...@apache.org.
DRILL-6099: Push limit past flatten(project) without pushdown into scan

closes #1096


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/6a55b2b2
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/6a55b2b2
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/6a55b2b2

Branch: refs/heads/master
Commit: 6a55b2b21e047ba44f8f2d19381f18ae44263e26
Parents: 6af651f
Author: Gautam Parai <gp...@maprtech.com>
Authored: Thu Jan 18 15:46:42 2018 -0800
Committer: Arina Ielchiieva <ar...@gmail.com>
Committed: Sat Mar 3 19:47:50 2018 +0200

----------------------------------------------------------------------
 .../apache/drill/exec/planner/PlannerPhase.java |  8 ++-
 .../exec/planner/common/DrillRelOptUtil.java    | 63 ++++++++++++++++++++
 .../exec/planner/logical/DrillLimitRel.java     |  6 +-
 .../logical/DrillPushLimitToScanRule.java       | 41 +++++++++----
 .../planner/sql/handlers/FindLimit0Visitor.java | 20 +------
 .../impl/flatten/TestFlattenPlanning.java       |  7 +++
 6 files changed, 112 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/6a55b2b2/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java
index 18dfb35..f46a7ee 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java
@@ -88,7 +88,7 @@ import java.util.List;
 public enum PlannerPhase {
   //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillRuleSets.class);
 
-  LOGICAL_PRUNE_AND_JOIN("Loigcal Planning (with join and partition pruning)") {
+  LOGICAL_PRUNE_AND_JOIN("Logical Planning (with join and partition pruning)") {
     public RuleSet getRules(OptimizerRulesContext context, Collection<StoragePlugin> plugins) {
       return PlannerPhase.mergedRuleSets(
           getDrillBasicRules(context),
@@ -274,6 +274,7 @@ public enum PlannerPhase {
        */
       DrillPushProjectPastFilterRule.INSTANCE,
       DrillPushProjectPastJoinRule.INSTANCE,
+
       // Due to infinite loop in planning (DRILL-3257), temporarily disable this rule
       //DrillProjectSetOpTransposeRule.INSTANCE,
       RuleInstance.PROJECT_WINDOW_TRANSPOSE_RULE,
@@ -342,8 +343,9 @@ public enum PlannerPhase {
             PruneScanRule.getDirFilterOnScan(optimizerRulesContext),
             ParquetPruneScanRule.getFilterOnProjectParquet(optimizerRulesContext),
             ParquetPruneScanRule.getFilterOnScanParquet(optimizerRulesContext),
-            DrillPushLimitToScanRule.LIMIT_ON_SCAN,
-            DrillPushLimitToScanRule.LIMIT_ON_PROJECT
+            // Include LIMIT_ON_PROJECT since LIMIT_ON_SCAN may not work without it
+            DrillPushLimitToScanRule.LIMIT_ON_PROJECT,
+            DrillPushLimitToScanRule.LIMIT_ON_SCAN
         )
         .build();
 

http://git-wip-us.apache.org/repos/asf/drill/blob/6a55b2b2/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillRelOptUtil.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillRelOptUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillRelOptUtil.java
index 91c33bd..d5c8d94 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillRelOptUtil.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillRelOptUtil.java
@@ -30,9 +30,11 @@ import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexCall;
 import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexVisitor;
 import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.util.Pair;
@@ -224,4 +226,65 @@ public abstract class DrillRelOptUtil {
     }
   }
 
+  public static boolean isLimit0(RexNode fetch) {
+    if (fetch != null && fetch.isA(SqlKind.LITERAL)) {
+      RexLiteral l = (RexLiteral) fetch;
+      switch (l.getTypeName()) {
+        case BIGINT:
+        case INTEGER:
+        case DECIMAL:
+          if (((long) l.getValue2()) == 0) {
+            return true;
+          }
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Find whether the given project rel can produce non-scalar output (hence unknown rowcount). This
+   * would happen if the project has a flatten
+   * @param project : The project rel
+   * @return : Return true if the rowcount is unknown. Otherwise, false.
+   */
+  public static boolean isProjectOutputRowcountUnknown(RelNode project) {
+    assert project instanceof Project : "Rel is NOT an instance of project!";
+    for (RexNode rex : project.getChildExps()) {
+      if (rex instanceof RexCall) {
+        if ("flatten".equals(((RexCall) rex).getOperator().getName().toLowerCase())) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Find whether the given project rel has unknown output schema. This would happen if the
+   * project has CONVERT_FROMJSON which can only derive the schema after evaluation is performed
+   * @param project : The project rel
+   * @return : Return true if the project output schema is unknown. Otherwise, false.
+   */
+  public static boolean isProjectOutputSchemaUnknown(RelNode project) {
+    assert project instanceof Project : "Rel is NOT an instance of project!";
+    try {
+      RexVisitor<Void> visitor =
+          new RexVisitorImpl<Void>(true) {
+            public Void visitCall(RexCall call) {
+              if ("convert_fromjson".equals(call.getOperator().getName().toLowerCase())) {
+                throw new Util.FoundOne(call); /* throw exception to interrupt tree walk (this is similar to
+                                              other utility methods in RexUtil.java */
+              }
+              return super.visitCall(call);
+            }
+          };
+      for (RexNode rex : ((Project) project).getProjects()) {
+        rex.accept(visitor);
+      }
+    } catch (Util.FoundOne e) {
+      Util.swallow(e, null);
+      return true;
+    }
+    return false;
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/6a55b2b2/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillLimitRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillLimitRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillLimitRel.java
index 9faf070..bef8b2b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillLimitRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillLimitRel.java
@@ -38,9 +38,13 @@ public class DrillLimitRel extends DrillLimitRelBase implements DrillRel {
     super(cluster, traitSet, child, offset, fetch);
   }
 
+  public DrillLimitRel(RelOptCluster cluster, RelTraitSet traitSet, RelNode child, RexNode offset, RexNode fetch, boolean pushDown) {
+    super(cluster, traitSet, child, offset, fetch, pushDown);
+  }
+
   @Override
   public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
-    return new DrillLimitRel(getCluster(), traitSet, sole(inputs), offset, fetch);
+    return new DrillLimitRel(getCluster(), traitSet, sole(inputs), offset, fetch, isPushDown());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/6a55b2b2/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushLimitToScanRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushLimitToScanRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushLimitToScanRule.java
index 9c06897..068252d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushLimitToScanRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushLimitToScanRule.java
@@ -23,7 +23,14 @@ import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptRuleOperand;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.util.Util;
 import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.planner.common.DrillRelOptUtil;
 
 public abstract class DrillPushLimitToScanRule extends RelOptRule {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillPushLimitToScanRule.class);
@@ -55,18 +62,21 @@ public abstract class DrillPushLimitToScanRule extends RelOptRule {
     }
   };
 
-  public static DrillPushLimitToScanRule LIMIT_ON_PROJECT =
-      new DrillPushLimitToScanRule(
-          RelOptHelper.some(DrillLimitRel.class, RelOptHelper.some(
-              DrillProjectRel.class, RelOptHelper.any(DrillScanRel.class))),
-          "DrillPushLimitToScanRule_LimitOnProject") {
+  public static DrillPushLimitToScanRule LIMIT_ON_PROJECT = new DrillPushLimitToScanRule(
+      RelOptHelper.some(DrillLimitRel.class, RelOptHelper.any(DrillProjectRel.class)), "DrillPushLimitToScanRule_LimitOnProject") {
     @Override
     public boolean matches(RelOptRuleCall call) {
       DrillLimitRel limitRel = call.rel(0);
-      DrillScanRel scanRel = call.rel(2);
-      // For now only applies to Parquet. And pushdown only apply limit but not offset,
+      DrillProjectRel projectRel = call.rel(1);
+      // pushdown only apply limit but not offset,
       // so if getFetch() return null no need to run this rule.
-      if (scanRel.getGroupScan().supportsLimitPushdown() && (limitRel.getFetch() != null)) {
+      // Do not push across Project containing CONVERT_FROMJSON for limit 0 queries. For limit 0 queries, this would
+      // mess up the schema since Convert_FromJson() is different from other regular functions in that it only knows
+      // the output schema after evaluation is performed. When input has 0 row, Drill essentially does not have a way
+      // to know the output type.
+      if (!limitRel.isPushDown() && (limitRel.getFetch() != null)
+          && (!DrillRelOptUtil.isLimit0(limitRel.getFetch())
+            || !DrillRelOptUtil.isProjectOutputSchemaUnknown(projectRel))) {
         return true;
       }
       return false;
@@ -76,12 +86,20 @@ public abstract class DrillPushLimitToScanRule extends RelOptRule {
     public void onMatch(RelOptRuleCall call) {
       DrillLimitRel limitRel = call.rel(0);
       DrillProjectRel projectRel = call.rel(1);
-      DrillScanRel scanRel = call.rel(2);
-      doOnMatch(call, limitRel, scanRel, projectRel);
+      RelNode child = projectRel.getInput();
+      final RelNode limitUnderProject = limitRel.copy(limitRel.getTraitSet(), ImmutableList.of(child));
+      final RelNode newProject = projectRel.copy(projectRel.getTraitSet(), ImmutableList.of(limitUnderProject));
+      if (DrillRelOptUtil.isProjectOutputRowcountUnknown(projectRel)) {
+        //Preserve limit above the project since Flatten can produce more rows. Also mark it so we do not fire the rule again.
+        final RelNode limitAboveProject = new DrillLimitRel(limitRel.getCluster(), limitRel.getTraitSet(), newProject,
+            limitRel.getOffset(), limitRel.getFetch(), true);
+        call.transformTo(limitAboveProject);
+      } else {
+        call.transformTo(newProject);
+      }
     }
   };
 
-
   protected void doOnMatch(RelOptRuleCall call, DrillLimitRel limitRel, DrillScanRel scanRel, DrillProjectRel projectRel){
     try {
       final int rowCountRequested = (int) limitRel.getRows();
@@ -113,6 +131,5 @@ public abstract class DrillPushLimitToScanRule extends RelOptRule {
     }  catch (Exception e) {
       logger.warn("Exception while using the pruned partitions.", e);
     }
-
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/6a55b2b2/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/FindLimit0Visitor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/FindLimit0Visitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/FindLimit0Visitor.java
index 166c350..03d5f75 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/FindLimit0Visitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/FindLimit0Visitor.java
@@ -41,6 +41,7 @@ import org.apache.drill.exec.expr.TypeHelper;
 import org.apache.drill.exec.ops.OperatorContext;
 import org.apache.drill.exec.physical.base.ScanStats;
 import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.planner.common.DrillRelOptUtil;
 import org.apache.drill.exec.planner.logical.DrillDirectScanRel;
 import org.apache.drill.exec.planner.logical.DrillLimitRel;
 import org.apache.drill.exec.planner.logical.DrillRel;
@@ -135,24 +136,9 @@ public class FindLimit0Visitor extends RelShuttleImpl {
     return contains;
   }
 
-  private static boolean isLimit0(RexNode fetch) {
-    if (fetch != null && fetch.isA(SqlKind.LITERAL)) {
-      RexLiteral l = (RexLiteral) fetch;
-      switch (l.getTypeName()) {
-      case BIGINT:
-      case INTEGER:
-      case DECIMAL:
-        if (((long) l.getValue2()) == 0) {
-          return true;
-        }
-      }
-    }
-    return false;
-  }
-
   @Override
   public RelNode visit(LogicalSort sort) {
-    if (isLimit0(sort.fetch)) {
+    if (DrillRelOptUtil.isLimit0(sort.fetch)) {
       contains = true;
       return sort;
     }
@@ -163,7 +149,7 @@ public class FindLimit0Visitor extends RelShuttleImpl {
   @Override
   public RelNode visit(RelNode other) {
     if (other instanceof DrillLimitRel) {
-      if (isLimit0(((DrillLimitRel) other).getFetch())) {
+      if (DrillRelOptUtil.isLimit0(((DrillLimitRel) other).getFetch())) {
         contains = true;
         return other;
       }

http://git-wip-us.apache.org/repos/asf/drill/blob/6a55b2b2/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/flatten/TestFlattenPlanning.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/flatten/TestFlattenPlanning.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/flatten/TestFlattenPlanning.java
index 1a5117f..0a28d69 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/flatten/TestFlattenPlanning.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/flatten/TestFlattenPlanning.java
@@ -63,5 +63,12 @@ public class TestFlattenPlanning extends PlanTestBase {
     PlanTestBase.testPlanMatchingPatterns(query, expectedPlans, excludedPlans);
   }
 
+  @Test // DRILL-6099 : push limit past flatten(project)
+  public void testLimitPushdownPastFlatten() throws Exception {
+    final String query = "select rownum, flatten(complex) comp from cp.`store/json/test_flatten_mappify2.json` limit 1";
+    final String[] expectedPatterns = {".*Limit\\(fetch=\\[1\\]\\).*",".*Flatten.*",".*Limit\\(fetch=\\[1\\]\\).*"};
+    final String[] excludedPatterns = null;
+    PlanTestBase.testPlanMatchingPatterns(query, expectedPatterns, excludedPatterns);
+  }
 
 }