You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by GitBox <gi...@apache.org> on 2018/07/10 07:24:27 UTC

[GitHub] okalinin closed pull request #1368: DRILL-5797: use Parquet new reader more often

okalinin closed pull request #1368: DRILL-5797: use Parquet new reader more often
URL: https://github.com/apache/drill/pull/1368
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/common/pom.xml b/common/pom.xml
index 5d54534ea2..a8cab1075d 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -53,13 +53,11 @@
     <dependency>
       <groupId>com.typesafe</groupId>
       <artifactId>config</artifactId>
-      <version>1.0.0</version>
     </dependency>
 
     <dependency>
       <groupId>org.apache.commons</groupId>
       <artifactId>commons-lang3</artifactId>
-      <version>3.1</version>
     </dependency>
 
     <dependency>
@@ -119,6 +117,11 @@
 
   <build>
     <plugins>
+      <plugin>
+        <!-- Creating a test artifact because javadoc needs to be able to find test classes -->
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+      </plugin>
       <plugin>
         <groupId>org.codehaus.mojo</groupId>
         <artifactId>exec-maven-plugin</artifactId>
diff --git a/common/src/main/java/org/apache/drill/common/CatastrophicFailure.java b/common/src/main/java/org/apache/drill/common/CatastrophicFailure.java
index 18e5747843..6a45daa09a 100644
--- a/common/src/main/java/org/apache/drill/common/CatastrophicFailure.java
+++ b/common/src/main/java/org/apache/drill/common/CatastrophicFailure.java
@@ -30,7 +30,7 @@ private CatastrophicFailure() {
    * Exit the VM as we hit a catastrophic failure.
    * @param e
    *          The Throwable that occurred
-   * @param name
+   * @param message
    *          A descriptive message
    * @param code
    *          An error code to exit the JVM with.
diff --git a/common/src/main/java/org/apache/drill/common/DrillAutoCloseables.java b/common/src/main/java/org/apache/drill/common/DrillAutoCloseables.java
index c1132d0372..3cc092188e 100644
--- a/common/src/main/java/org/apache/drill/common/DrillAutoCloseables.java
+++ b/common/src/main/java/org/apache/drill/common/DrillAutoCloseables.java
@@ -28,13 +28,13 @@ private DrillAutoCloseables() {
   }
 
   /**
-   * close() an {@see java.lang.AutoCloseable} without throwing a (checked)
-   * {@see java.lang.Exception}. This wraps the close() call with a
+   * close() an {@link java.lang.AutoCloseable} without throwing a (checked)
+   * {@link java.lang.Exception}. This wraps the close() call with a
    * try-catch that will rethrow an Exception wrapped with a
-   * {@see java.lang.RuntimeException}, providing a way to call close()
+   * {@link java.lang.RuntimeException}, providing a way to call close()
    * without having to do the try-catch everywhere or propagate the Exception.
    *
-   * @param closeable the AutoCloseable to close; may be null
+   * @param autoCloseable the AutoCloseable to close; may be null
    * @throws RuntimeException if an Exception occurs; the Exception is
    *   wrapped by the RuntimeException
    */
diff --git a/common/src/main/java/org/apache/drill/common/config/CommonConstants.java b/common/src/main/java/org/apache/drill/common/config/CommonConstants.java
index 1b5fb29c01..e203972b8e 100644
--- a/common/src/main/java/org/apache/drill/common/config/CommonConstants.java
+++ b/common/src/main/java/org/apache/drill/common/config/CommonConstants.java
@@ -31,4 +31,7 @@
   /** Override configuration file name.  (Classpath resource pathname.) */
   String CONFIG_OVERRIDE_RESOURCE_PATHNAME = "drill-override.conf";
 
+  /** Override plugins configs file name.  (Classpath resource pathname.) */
+  String STORAGE_PLUGINS_OVERRIDE_CONF = "storage-plugins-override.conf";
+
 }
diff --git a/common/src/main/java/org/apache/drill/common/config/DrillConfig.java b/common/src/main/java/org/apache/drill/common/config/DrillConfig.java
index 66058643da..7211f19363 100644
--- a/common/src/main/java/org/apache/drill/common/config/DrillConfig.java
+++ b/common/src/main/java/org/apache/drill/common/config/DrillConfig.java
@@ -261,7 +261,7 @@ private static DrillConfig create(String overrideFileResourcePathname,
     final String className = getString(location);
     if (className == null) {
       throw new DrillConfigurationException(String.format(
-          "No class defined at location '%s'. Expected a definition of the class []",
+          "No class defined at location '%s'. Expected a definition of the class [%s]",
           location, clazz.getCanonicalName()));
     }
 
diff --git a/common/src/main/java/org/apache/drill/common/scanner/ClassPathScanner.java b/common/src/main/java/org/apache/drill/common/scanner/ClassPathScanner.java
index fa64ac081e..909e8110df 100644
--- a/common/src/main/java/org/apache/drill/common/scanner/ClassPathScanner.java
+++ b/common/src/main/java/org/apache/drill/common/scanner/ClassPathScanner.java
@@ -51,7 +51,6 @@
 import com.google.common.base.Stopwatch;
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Multimap;
-import com.google.common.collect.Sets;
 
 import javassist.bytecode.AccessFlag;
 import javassist.bytecode.AnnotationsAttribute;
@@ -75,13 +74,13 @@
 
 /**
  * Classpath scanning utility.
- * The classpath should be scanned once at startup from a DrillConfig instance. {@see ClassPathScanner#fromPrescan(DrillConfig)}
+ * The classpath should be scanned once at startup from a DrillConfig instance. {@link ClassPathScanner#fromPrescan(DrillConfig)}
  * The DrillConfig provides:
- *  - the list of packages to scan. (drill.classpath.scanning.packages) {@see CommonConstants#IMPLEMENTATIONS_SCAN_PACKAGES}
- *  - the list of base classes to scan for implementations. (drill.classpath.scanning.base.classes) {@see CommonConstants#IMPLEMENTATIONS_SCAN_CLASSES}
- *  - the list of annotations to scan for. (drill.classpath.scanning.annotations) {@see CommonConstants#IMPLEMENTATIONS_SCAN_ANNOTATIONS}
+ *  - the list of packages to scan. (drill.classpath.scanning.packages) {@link ClassPathScanner#IMPLEMENTATIONS_SCAN_PACKAGES}
+ *  - the list of base classes to scan for implementations. (drill.classpath.scanning.base.classes) {@link ClassPathScanner#IMPLEMENTATIONS_SCAN_CLASSES}
+ *  - the list of annotations to scan for. (drill.classpath.scanning.annotations) {@link ClassPathScanner#IMPLEMENTATIONS_SCAN_ANNOTATIONS}
  * Only the class directories and jars containing a drill-module.conf will be scanned.
- * Drill core packages are scanned at build time and the result is saved in a JSON file. {@see ClassPathScanner#FUNCTION_REGISTRY_FILE}
+ * Drill core packages are scanned at build time and the result is saved in a JSON file.
  * At runtime only the locations that have not been scanned yet will be scanned.
  */
 public final class ClassPathScanner {
@@ -320,15 +319,12 @@ public void scan(final Object cls) {
    *           to scan for (relative to specified class loaders' classpath roots)
    * @param  returnRootPathname  whether to collect classpath root portion of
    *           URL for each resource instead of full URL of each resource
-   * @param  classLoaders  set of class loaders in which to look up resource;
-   *           none (empty array) to specify to use current thread's context
-   *           class loader and {@link Reflections}'s class loader
    * @returns  ...; empty set if none
    */
   public static Set<URL> forResource(final String resourcePathname, final boolean returnRootPathname) {
     logger.debug("Scanning classpath for resources with pathname \"{}\".",
                  resourcePathname);
-    final Set<URL> resultUrlSet = Sets.newHashSet();
+    final Set<URL> resultUrlSet = new HashSet<>();
     final ClassLoader classLoader = ClassPathScanner.class.getClassLoader();
     try {
       final Enumeration<URL> resourceUrls = classLoader.getResources(resourcePathname);
diff --git a/common/src/main/java/org/apache/drill/common/scanner/persistence/AnnotationDescriptor.java b/common/src/main/java/org/apache/drill/common/scanner/persistence/AnnotationDescriptor.java
index 466922950f..67df91e70a 100644
--- a/common/src/main/java/org/apache/drill/common/scanner/persistence/AnnotationDescriptor.java
+++ b/common/src/main/java/org/apache/drill/common/scanner/persistence/AnnotationDescriptor.java
@@ -148,6 +148,8 @@ public Object invoke(Object proxy, Method method, Object[] args) throws Throwabl
           return enumValue;
         } else if (c.equals(boolean.class)) {
           return Boolean.valueOf(value);
+        } else if (c.equals(int.class)) {
+          return Integer.valueOf(value);
         }
         throw new UnsupportedOperationException(c.toString());
       }
diff --git a/common/src/main/java/org/apache/drill/exec/util/ActionOnFile.java b/common/src/main/java/org/apache/drill/exec/util/ActionOnFile.java
new file mode 100644
index 0000000000..cca1e771d7
--- /dev/null
+++ b/common/src/main/java/org/apache/drill/exec/util/ActionOnFile.java
@@ -0,0 +1,87 @@
+/*
+ * 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.util;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URL;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+
+/**
+ * It defines possible actions on the file and performs the necessary action
+ */
+public enum ActionOnFile {
+
+  /**
+   * No action will be performed
+   */
+  NONE {
+    @Override
+    public void action(URL url) { }
+  },
+
+  /**
+   * Rename the file by adding current timestamp value with "yyyyMMdd_HHmmss" format before last dot of original file name<p>
+   * Example:<br>
+   * Original file name: "storage-plugins-override.conf"<br>
+   * New file name: "storage-plugins-override-20180703_033354.conf"
+   */
+  RENAME {
+    @Override
+    public void action(URL url) {
+      String fileName = url.getFile();
+      File file = new File(url.getPath());
+      String currentDateTime = new SimpleDateFormat("yyyyMMdd_HHmmss").format(new Date());
+      String newFileName = new StringBuilder(fileName)
+          .insert(fileName.lastIndexOf("."), "-" + currentDateTime)
+          .toString();
+      Path filePath = file.toPath();
+      try {
+        Files.move(filePath, filePath.resolveSibling(newFileName));
+      } catch (IOException e) {
+        logger.error("There was an error during file {} rename.", fileName, e);
+      }
+    }
+  },
+
+  /**
+   * It removes the file
+   */
+  REMOVE {
+    @Override
+    public void action(URL url) {
+      File file = new File(url.getPath());
+      try {
+        Files.delete(file.toPath());
+      } catch (IOException e) {
+        logger.error("There was an error during file {} removing.", url.getFile(), e);
+      }
+    }
+  };
+
+  private static final org.slf4j.Logger logger =  org.slf4j.LoggerFactory.getLogger(ActionOnFile.class);
+
+  /**
+   * This is an action which should be performed on the file
+   * @param url the file URL
+   */
+  public abstract void action(URL url);
+}
diff --git a/common/src/test/java/org/apache/drill/test/Drill2130CommonHamcrestConfigurationTest.java b/common/src/test/java/org/apache/drill/test/Drill2130CommonHamcrestConfigurationTest.java
index ee814881cb..a747966c3c 100644
--- a/common/src/test/java/org/apache/drill/test/Drill2130CommonHamcrestConfigurationTest.java
+++ b/common/src/test/java/org/apache/drill/test/Drill2130CommonHamcrestConfigurationTest.java
@@ -24,6 +24,7 @@
 import static org.hamcrest.CoreMatchers.equalTo;
 
 public class Drill2130CommonHamcrestConfigurationTest {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Drill2130CommonHamcrestConfigurationTest.class);
 
   @SuppressWarnings("unused")
   private org.hamcrest.MatcherAssert forCompileTimeCheckForNewEnoughHamcrest;
@@ -38,8 +39,7 @@ public void testJUnitHamcrestMatcherFailureWorks() {
              + "  Got NoSuchMethodError;  e: " + e );
     }
     catch ( AssertionError e ) {
-      System.out.println( "Class path seems fine re new JUnit vs. old Hamcrest."
-                          + " (Got AssertionError, not NoSuchMethodError.)" );
+      logger.info("Class path seems fine re new JUnit vs. old Hamcrest. (Got AssertionError, not NoSuchMethodError.)");
     }
   }
 
diff --git a/common/src/test/java/org/apache/drill/test/DrillTest.java b/common/src/test/java/org/apache/drill/test/DrillTest.java
index 28544ab3c3..2ea4932e90 100644
--- a/common/src/test/java/org/apache/drill/test/DrillTest.java
+++ b/common/src/test/java/org/apache/drill/test/DrillTest.java
@@ -17,21 +17,17 @@
  */
 package org.apache.drill.test;
 
-import java.io.PrintStream;
 import java.lang.management.BufferPoolMXBean;
 import java.lang.management.ManagementFactory;
 import java.lang.management.MemoryMXBean;
 import java.util.List;
 
-import org.apache.commons.io.output.NullOutputStream;
 import org.apache.drill.common.util.DrillStringUtils;
 import org.junit.AfterClass;
-import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.rules.DisableOnDebug;
 import org.junit.rules.ExpectedException;
-import org.junit.rules.TestName;
 import org.junit.rules.TestRule;
 import org.junit.rules.TestWatcher;
 import org.junit.runner.Description;
@@ -68,32 +64,6 @@
    */
   @Rule public final ExpectedException thrownException = ExpectedException.none();
 
-  @Rule public TestName TEST_NAME = new TestName();
-
-  /**
-   * Option to cause tests to produce verbose output. Many tests provide
-   * detailed information to stdout when enabled. To enable:
-   * <p>
-   * <tt>java ... -Dtest.verbose=true ...</tt>
-   */
-  public static final String VERBOSE_OUTPUT = "test.verbose";
-
-  protected static final boolean verbose = Boolean.parseBoolean(System.getProperty(VERBOSE_OUTPUT));
-
-  /**
-   * Output destination for verbose test output. Rather than using
-   * <tt>System.out</tt>, use <tt>DrillTest.out</tt>. Output will
-   * automagically be routed to the bit bucket unless the
-   * {@link #VERBOSE_OUTPUT} flag is set.
-   */
-
-  public static final PrintStream out = verbose ? System.out : new PrintStream(new NullOutputStream());
-
-  @Before
-  public void printID() throws Exception {
-    System.out.printf("Running %s#%s\n", getClass().getName(), TEST_NAME.getMethodName());
-  }
-
   @BeforeClass
   public static void initDrillTest() throws Exception {
     memWatcher = new MemWatcher();
@@ -195,16 +165,4 @@ public long getMemNonHeap() {
       return memoryBean.getNonHeapMemoryUsage().getUsed();
     }
   }
-
-  /**
-   * Reports whether verbose output has been selected for this test run.
-   *
-   * @return <tt>true</tt> if verbose output is wanted (test is likely running
-   * in a debugger), <tt>false</tt> if verbose output is to be suppressed
-   * (test is likely running in a batch Maven build).
-   */
-
-  public static boolean verbose( ) {
-    return verbose;
-  }
 }
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/TableFormatPlugin.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/TableFormatPlugin.java
index 0d983bd38c..be86b4c7c7 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/TableFormatPlugin.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/TableFormatPlugin.java
@@ -31,6 +31,7 @@
 import org.apache.drill.exec.physical.base.AbstractWriter;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.AbstractStoragePlugin;
 import org.apache.drill.exec.store.StoragePluginOptimizerRule;
 import org.apache.drill.exec.store.dfs.FileSystemConfig;
 import org.apache.drill.exec.store.dfs.FileSystemPlugin;
@@ -45,20 +46,20 @@
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory
       .getLogger(TableFormatPlugin.class);
 
-  private final FileSystemConfig storageConfig;
+  private final StoragePluginConfig storageConfig;
   private final TableFormatPluginConfig config;
   private final Configuration fsConf;
   private final DrillbitContext context;
   private final String name;
 
-  private volatile FileSystemPlugin storagePlugin;
+  private volatile AbstractStoragePlugin storagePlugin;
   private final MapRFileSystem maprfs;
 
   protected TableFormatPlugin(String name, DrillbitContext context, Configuration fsConf,
       StoragePluginConfig storageConfig, TableFormatPluginConfig formatConfig) {
     this.context = context;
     this.config = formatConfig;
-    this.storageConfig = (FileSystemConfig) storageConfig;
+    this.storageConfig = storageConfig;
     this.fsConf = fsConf;
     this.name = name == null ? "maprdb" : name;
     try {
@@ -119,10 +120,10 @@ public String getName() {
     return name;
   }
 
-  public synchronized FileSystemPlugin getStoragePlugin() {
+  public synchronized AbstractStoragePlugin getStoragePlugin() {
     if (this.storagePlugin == null) {
       try {
-        this.storagePlugin = (FileSystemPlugin) (context.getStorage().getPlugin(storageConfig));
+        this.storagePlugin = (AbstractStoragePlugin) context.getStorage().getPlugin(storageConfig);
       } catch (ExecutionSetupException e) {
         throw new RuntimeException(e);
       }
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatMatcher.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatMatcher.java
index 4ce6b7defc..d4978b95d8 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatMatcher.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatMatcher.java
@@ -18,8 +18,8 @@
 package org.apache.drill.exec.store.mapr.db;
 
 import java.io.IOException;
-import java.util.List;
 
+import com.mapr.fs.MapRFileStatus;
 import com.mapr.fs.tables.TableProperties;
 import org.apache.drill.exec.planner.logical.DrillTable;
 import org.apache.drill.exec.planner.logical.DynamicDrillTable;
@@ -31,7 +31,6 @@
 import org.apache.drill.exec.store.mapr.TableFormatMatcher;
 import org.apache.drill.exec.store.mapr.TableFormatPlugin;
 
-import com.mapr.fs.MapRFileStatus;
 import org.apache.drill.exec.store.mapr.db.binary.MapRDBBinaryTable;
 import org.apache.hadoop.fs.Path;
 
@@ -54,19 +53,16 @@ protected boolean isSupportedTable(MapRFileStatus status) throws IOException {
   public DrillTable isReadable(DrillFileSystem fs,
                                FileSelection selection, FileSystemPlugin fsPlugin,
                                String storageEngineName, SchemaConfig schemaConfig) throws IOException {
-
     if (isFileReadable(fs, selection.getFirstPath(fs))) {
-      List<String> files = selection.getFiles();
-      assert (files.size() == 1);
-      String tableName = files.get(0);
-      TableProperties props = getFormatPlugin().getMaprFS().getTableProperties(new Path(tableName));
-
+      MapRDBFormatPlugin mapRDBFormatPlugin = (MapRDBFormatPlugin) getFormatPlugin();
+      String tableName = mapRDBFormatPlugin.getTableName(selection);
+      TableProperties props = mapRDBFormatPlugin.getMaprFS().getTableProperties(new Path(tableName));
       if (props.getAttr().getJson()) {
         return new DynamicDrillTable(fsPlugin, storageEngineName, schemaConfig.getUserName(),
-            new FormatSelection(getFormatPlugin().getConfig(), selection));
+            new FormatSelection(mapRDBFormatPlugin.getConfig(), selection));
       } else {
-        FormatSelection formatSelection = new FormatSelection(getFormatPlugin().getConfig(), selection);
-        return new MapRDBBinaryTable(storageEngineName, fsPlugin, (MapRDBFormatPlugin) getFormatPlugin(), formatSelection);
+        FormatSelection formatSelection = new FormatSelection(mapRDBFormatPlugin.getConfig(), selection);
+        return new MapRDBBinaryTable(storageEngineName, fsPlugin, mapRDBFormatPlugin, formatSelection);
       }
     }
     return null;
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBGroupScan.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBGroupScan.java
index effdcdebe8..927bd70c35 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBGroupScan.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBGroupScan.java
@@ -35,8 +35,8 @@
 import org.apache.drill.exec.physical.EndpointAffinity;
 import org.apache.drill.exec.physical.base.AbstractGroupScan;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.store.AbstractStoragePlugin;
 import org.apache.drill.exec.store.dfs.FileSystemConfig;
-import org.apache.drill.exec.store.dfs.FileSystemPlugin;
 
 import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
@@ -49,7 +49,7 @@
 public abstract class MapRDBGroupScan extends AbstractGroupScan {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MapRDBGroupScan.class);
 
-  protected FileSystemPlugin storagePlugin;
+  protected AbstractStoragePlugin storagePlugin;
 
   protected MapRDBFormatPlugin formatPlugin;
 
@@ -84,7 +84,7 @@ public MapRDBGroupScan(MapRDBGroupScan that) {
     this.filterPushedDown = that.filterPushedDown;
   }
 
-  public MapRDBGroupScan(FileSystemPlugin storagePlugin,
+  public MapRDBGroupScan(AbstractStoragePlugin storagePlugin,
       MapRDBFormatPlugin formatPlugin, List<SchemaPath> columns, String userName) {
     super(userName);
     this.storagePlugin = storagePlugin;
@@ -254,7 +254,7 @@ public FileSystemConfig getStorageConfig() {
   }
 
   @JsonIgnore
-  public FileSystemPlugin getStoragePlugin(){
+  public AbstractStoragePlugin getStoragePlugin(){
     return storagePlugin;
   }
 
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBPushFilterIntoScan.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBPushFilterIntoScan.java
index 8655f5b2fd..e2ba23844c 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBPushFilterIntoScan.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBPushFilterIntoScan.java
@@ -144,7 +144,7 @@ protected void doPushFilterIntoJsonGroupScan(RelOptRuleCall call,
     final ScanPrel newScanPrel = ScanPrel.create(scan, filter.getTraitSet(), newGroupsScan, scan.getRowType());
 
     // Depending on whether is a project in the middle, assign either scan or copy of project to childRel.
-    final RelNode childRel = project == null ? newScanPrel : project.copy(project.getTraitSet(), ImmutableList.of((RelNode)newScanPrel));;
+    final RelNode childRel = project == null ? newScanPrel : project.copy(project.getTraitSet(), ImmutableList.of((RelNode)newScanPrel));
 
     if (jsonConditionBuilder.isAllExpressionsConverted()) {
         /*
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/BinaryTableGroupScan.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/BinaryTableGroupScan.java
index 16b979e51d..216f05e7c3 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/BinaryTableGroupScan.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/BinaryTableGroupScan.java
@@ -30,9 +30,9 @@
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.ScanStats;
 import org.apache.drill.exec.physical.base.ScanStats.GroupScanProperty;
+import org.apache.drill.exec.store.AbstractStoragePlugin;
 import org.apache.drill.exec.store.StoragePluginRegistry;
 import org.apache.drill.exec.store.dfs.FileSystemConfig;
-import org.apache.drill.exec.store.dfs.FileSystemPlugin;
 import org.apache.drill.exec.store.hbase.DrillHBaseConstants;
 import org.apache.drill.exec.store.hbase.HBaseScanSpec;
 import org.apache.drill.exec.store.hbase.HBaseUtils;
@@ -78,19 +78,19 @@ public BinaryTableGroupScan(@JsonProperty("userName") final String userName,
                               @JsonProperty("columns") List<SchemaPath> columns,
                               @JacksonInject StoragePluginRegistry pluginRegistry) throws IOException, ExecutionSetupException {
     this (userName,
-          (FileSystemPlugin) pluginRegistry.getPlugin(storagePluginConfig),
+          (AbstractStoragePlugin) pluginRegistry.getPlugin(storagePluginConfig),
           (MapRDBFormatPlugin) pluginRegistry.getFormatPlugin(storagePluginConfig, formatPluginConfig),
           scanSpec, columns);
   }
 
-  public BinaryTableGroupScan(String userName, FileSystemPlugin storagePlugin,
+  public BinaryTableGroupScan(String userName, AbstractStoragePlugin storagePlugin,
       MapRDBFormatPlugin formatPlugin, HBaseScanSpec scanSpec, List<SchemaPath> columns) {
     super(storagePlugin, formatPlugin, columns, userName);
     this.hbaseScanSpec = scanSpec;
     init();
   }
 
-  public BinaryTableGroupScan(String userName, FileSystemPlugin storagePlugin,
+  public BinaryTableGroupScan(String userName, AbstractStoragePlugin storagePlugin,
                               MapRDBFormatPlugin formatPlugin, HBaseScanSpec scanSpec,
                               List<SchemaPath> columns, MapRDBTableStats tableStats) {
     super(storagePlugin, formatPlugin, columns, userName);
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableGroupScan.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableGroupScan.java
index a1d7f9abff..975f1b8d98 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableGroupScan.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableGroupScan.java
@@ -30,9 +30,9 @@
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.ScanStats;
 import org.apache.drill.exec.physical.base.ScanStats.GroupScanProperty;
+import org.apache.drill.exec.store.AbstractStoragePlugin;
 import org.apache.drill.exec.store.StoragePluginRegistry;
 import org.apache.drill.exec.store.dfs.FileSystemConfig;
-import org.apache.drill.exec.store.dfs.FileSystemPlugin;
 import org.apache.drill.exec.store.mapr.db.MapRDBFormatPlugin;
 import org.apache.drill.exec.store.mapr.db.MapRDBFormatPluginConfig;
 import org.apache.drill.exec.store.mapr.db.MapRDBGroupScan;
@@ -71,12 +71,12 @@ public JsonTableGroupScan(@JsonProperty("userName") final String userName,
                             @JsonProperty("columns") List<SchemaPath> columns,
                             @JacksonInject StoragePluginRegistry pluginRegistry) throws IOException, ExecutionSetupException {
     this (userName,
-          (FileSystemPlugin) pluginRegistry.getPlugin(storagePluginConfig),
+          (AbstractStoragePlugin) pluginRegistry.getPlugin(storagePluginConfig),
           (MapRDBFormatPlugin) pluginRegistry.getFormatPlugin(storagePluginConfig, formatPluginConfig),
           scanSpec, columns);
   }
 
-  public JsonTableGroupScan(String userName, FileSystemPlugin storagePlugin,
+  public JsonTableGroupScan(String userName, AbstractStoragePlugin storagePlugin,
                             MapRDBFormatPlugin formatPlugin, JsonScanSpec scanSpec, List<SchemaPath> columns) {
     super(storagePlugin, formatPlugin, columns, userName);
     this.scanSpec = scanSpec;
diff --git a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/MaprDBTestsSuite.java b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/MaprDBTestsSuite.java
index f43fc74586..0c81a18444 100644
--- a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/MaprDBTestsSuite.java
+++ b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/MaprDBTestsSuite.java
@@ -70,7 +70,6 @@ public static void setupTests() throws Exception {
           // Sleep to allow table data to be flushed to tables.
           // Without this, the row count stats to return 0,
           // causing the planner to reject optimized plans.
-          System.out.println("Sleeping for 5 seconds to allow table flushes");
           Thread.sleep(5000);
 
           conf = HBaseTestsSuite.getConf();
diff --git a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/json/BaseJsonTest.java b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/json/BaseJsonTest.java
index 4b3ec5b835..ee32aa18ef 100644
--- a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/json/BaseJsonTest.java
+++ b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/json/BaseJsonTest.java
@@ -50,7 +50,6 @@ public static void tearDownAfterClass() throws Exception {
 
 
   protected List<QueryDataBatch> runHBaseSQLlWithResults(String sql) throws Exception {
-    System.out.println("Running query:\n" + sql);
     return testSqlWithResults(sql);
   }
 
@@ -65,5 +64,4 @@ private void printResultAndVerifyRowCount(List<QueryDataBatch> results, int expe
       Assert.assertEquals(expectedRowCount, rowCount);
     }
   }
-
 }
diff --git a/contrib/gis/pom.xml b/contrib/gis/pom.xml
index 30a0924133..96f694a65e 100644
--- a/contrib/gis/pom.xml
+++ b/contrib/gis/pom.xml
@@ -45,7 +45,11 @@
 			<artifactId>esri-geometry-api</artifactId>
 			<version>2.0.0</version>
 		</dependency>
-
+		<dependency>
+		    <groupId>org.osgeo</groupId>
+		    <artifactId>proj4j</artifactId>
+		    <version>0.1.0</version>
+		</dependency>
 		<!-- Test dependencies -->
 		<dependency>
 			<groupId>org.apache.drill.exec</groupId>
diff --git a/contrib/gis/sample-data/CA-cities-with-nulls.csv b/contrib/gis/sample-data/CA-cities-with-nulls.csv
new file mode 100644
index 0000000000..dcd67ecca3
--- /dev/null
+++ b/contrib/gis/sample-data/CA-cities-with-nulls.csv
@@ -0,0 +1,3 @@
+US,6,"Acampo",
+US,6,"Agnew",POINT(-121.9591252 37.3946626)
+US,6,"Agua Caliente",POINT(-122.4880366 38.3240804)
\ No newline at end of file
diff --git a/contrib/gis/sample-data/polygons.tsv b/contrib/gis/sample-data/polygons.tsv
new file mode 100644
index 0000000000..f70c922b1a
--- /dev/null
+++ b/contrib/gis/sample-data/polygons.tsv
@@ -0,0 +1,5 @@
+1	POLYGON((0 0, 0 1, 1 1, 1 0, 0 0))
+1	POLYGON((0 0, 0 -1, 1 -1, 1 0, 0 0))
+2	POLYGON((10 10, 10 11, 11 11, 11 10, 10 10))
+2	POLYGON((10 10, 10 9, 11 9, 11 10, 10 10))
+2 
diff --git a/contrib/gis/src/main/java/com/esri/core/geometry/VertexGeomAccessor.java b/contrib/gis/src/main/java/com/esri/core/geometry/VertexGeomAccessor.java
new file mode 100644
index 0000000000..c95b798758
--- /dev/null
+++ b/contrib/gis/src/main/java/com/esri/core/geometry/VertexGeomAccessor.java
@@ -0,0 +1,25 @@
+/*
+ * 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 com.esri.core.geometry;
+
+public class VertexGeomAccessor {
+  public static MultiVertexGeometry getVertexGeometry(Geometry geom){
+    return (MultiVertexGeometry) geom._getImpl();
+  }
+}
diff --git a/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STBuffer.java b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STBuffer.java
new file mode 100644
index 0000000000..a1d3af45f8
--- /dev/null
+++ b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STBuffer.java
@@ -0,0 +1,69 @@
+/*
+ * 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.fn.impl.gis;
+
+import javax.inject.Inject;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.Float8Holder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+
+import io.netty.buffer.DrillBuf;
+
+/*
+* Returns a geometry that represents all points whose distance from this Geometry
+* is less than or equal to radius
+*/
+@FunctionTemplate(name = "st_buffer", scope = FunctionTemplate.FunctionScope.SIMPLE,
+  nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+public class STBuffer implements DrillSimpleFunc {
+  @Param
+  VarBinaryHolder geom1Param;
+
+  @Param
+  Float8Holder bufferRadiusParam;
+
+  @Output
+  VarBinaryHolder out;
+
+  @Inject
+  DrillBuf buffer;
+
+  public void setup() {
+  }
+
+  public void eval() {
+    double bufferRadius = bufferRadiusParam.value;
+
+    com.esri.core.geometry.ogc.OGCGeometry geom1 = com.esri.core.geometry.ogc.OGCGeometry
+        .fromBinary(geom1Param.buffer.nioBuffer(geom1Param.start, geom1Param.end - geom1Param.start));
+
+    com.esri.core.geometry.ogc.OGCGeometry bufferedGeom = geom1.buffer(bufferRadius);
+
+    java.nio.ByteBuffer bufferedGeomBytes = bufferedGeom.asBinary();
+
+    int outputSize = bufferedGeomBytes.remaining();
+    buffer = out.buffer = buffer.reallocIfNeeded(outputSize);
+    out.start = 0;
+    out.end = outputSize;
+    buffer.setBytes(0, bufferedGeomBytes);
+  }
+}
diff --git a/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STContains.java b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STContains.java
new file mode 100644
index 0000000000..5204ce605c
--- /dev/null
+++ b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STContains.java
@@ -0,0 +1,66 @@
+/*
+ * 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.fn.impl.gis;
+
+import javax.inject.Inject;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.BitHolder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+
+import io.netty.buffer.DrillBuf;
+
+/*
+ Returns true if and only if no points of B lie in the exterior of A,
+ and at least one point of the interior of B lies in the interior of A.
+*/
+@FunctionTemplate(name = "st_contains", scope = FunctionTemplate.FunctionScope.SIMPLE,
+  nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+public class STContains implements DrillSimpleFunc {
+  @Param
+  VarBinaryHolder geom1Param;
+
+  @Param
+  VarBinaryHolder geom2Param;
+
+  @Output
+  BitHolder out;
+
+  @Inject
+  DrillBuf buffer;
+
+  public void setup() {
+  }
+
+  public void eval() {
+    com.esri.core.geometry.ogc.OGCGeometry geom1;
+    com.esri.core.geometry.ogc.OGCGeometry geom2;
+
+    geom1 = com.esri.core.geometry.ogc.OGCGeometry
+        .fromBinary(geom1Param.buffer.nioBuffer(geom1Param.start, geom1Param.end - geom1Param.start));
+    geom2 = com.esri.core.geometry.ogc.OGCGeometry
+        .fromBinary(geom2Param.buffer.nioBuffer(geom2Param.start, geom2Param.end - geom2Param.start));
+
+    int contains = geom1.contains(geom2) ? 1 : 0;
+
+    out.value = contains;
+  }
+}
diff --git a/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STCrosses.java b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STCrosses.java
new file mode 100644
index 0000000000..95d7aca803
--- /dev/null
+++ b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STCrosses.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.expr.fn.impl.gis;
+
+import javax.inject.Inject;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.BitHolder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+
+import io.netty.buffer.DrillBuf;
+
+/*
+ * Returns TRUE if the supplied geometries have some, but not all, interior points in common
+ */
+@FunctionTemplate(name = "st_crosses", scope = FunctionTemplate.FunctionScope.SIMPLE,
+  nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+public class STCrosses implements DrillSimpleFunc {
+  @Param
+  VarBinaryHolder geom1Param;
+
+  @Param
+  VarBinaryHolder geom2Param;
+
+  @Output
+  BitHolder out;
+
+  @Inject
+  DrillBuf buffer;
+
+  public void setup() {
+  }
+
+  public void eval() {
+    com.esri.core.geometry.ogc.OGCGeometry geom1;
+    com.esri.core.geometry.ogc.OGCGeometry geom2;
+
+    geom1 = com.esri.core.geometry.ogc.OGCGeometry
+        .fromBinary(geom1Param.buffer.nioBuffer(geom1Param.start, geom1Param.end - geom1Param.start));
+    geom2 = com.esri.core.geometry.ogc.OGCGeometry
+        .fromBinary(geom2Param.buffer.nioBuffer(geom2Param.start, geom2Param.end - geom2Param.start));
+
+    int crosses = geom1.crosses(geom2) ? 1 : 0;
+
+    out.value = crosses;
+  }
+}
diff --git a/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STDifference.java b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STDifference.java
new file mode 100644
index 0000000000..4906880e95
--- /dev/null
+++ b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STDifference.java
@@ -0,0 +1,70 @@
+/*
+ * 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.fn.impl.gis;
+
+import javax.inject.Inject;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+
+import io.netty.buffer.DrillBuf;
+
+/*
+ * Given geometries A and B, this function returns a geometry that represents
+ * the part of geometry A that does not intersect with geometry B
+ */
+@FunctionTemplate(name = "st_difference", scope = FunctionTemplate.FunctionScope.SIMPLE,
+  nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+public class STDifference implements DrillSimpleFunc {
+  @Param
+  VarBinaryHolder geom1Param;
+
+  @Param
+  VarBinaryHolder geom2Param;
+
+  @Output
+  VarBinaryHolder out;
+
+  @Inject
+  DrillBuf buffer;
+
+  public void setup() {
+  }
+
+  public void eval() {
+    com.esri.core.geometry.ogc.OGCGeometry geom1;
+    com.esri.core.geometry.ogc.OGCGeometry geom2;
+    geom1 = com.esri.core.geometry.ogc.OGCGeometry
+        .fromBinary(geom1Param.buffer.nioBuffer(geom1Param.start, geom1Param.end - geom1Param.start));
+    geom2 = com.esri.core.geometry.ogc.OGCGeometry
+        .fromBinary(geom2Param.buffer.nioBuffer(geom2Param.start, geom2Param.end - geom2Param.start));
+
+    com.esri.core.geometry.ogc.OGCGeometry diffGeom = geom1.difference(geom2);
+
+    java.nio.ByteBuffer bufferedGeomBytes = diffGeom.asBinary();
+
+    int outputSize = bufferedGeomBytes.remaining();
+    buffer = out.buffer = buffer.reallocIfNeeded(outputSize);
+    out.start = 0;
+    out.end = outputSize;
+    buffer.setBytes(0, bufferedGeomBytes);
+  }
+}
diff --git a/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STDisjoint.java b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STDisjoint.java
new file mode 100644
index 0000000000..8a34241481
--- /dev/null
+++ b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STDisjoint.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.expr.fn.impl.gis;
+
+import javax.inject.Inject;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.BitHolder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+
+import io.netty.buffer.DrillBuf;
+
+/*
+ * Returns TRUE if two Geometries do not "spatially intersect" - if they do not share any space
+ */
+@FunctionTemplate(name = "st_disjoint", scope = FunctionTemplate.FunctionScope.SIMPLE,
+  nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+public class STDisjoint implements DrillSimpleFunc {
+  @Param
+  VarBinaryHolder geom1Param;
+
+  @Param
+  VarBinaryHolder geom2Param;
+
+  @Output
+  BitHolder out;
+
+  @Inject
+  DrillBuf buffer;
+
+  public void setup() {
+  }
+
+  public void eval() {
+    com.esri.core.geometry.ogc.OGCGeometry geom1;
+    com.esri.core.geometry.ogc.OGCGeometry geom2;
+
+    geom1 = com.esri.core.geometry.ogc.OGCGeometry
+        .fromBinary(geom1Param.buffer.nioBuffer(geom1Param.start, geom1Param.end - geom1Param.start));
+    geom2 = com.esri.core.geometry.ogc.OGCGeometry
+        .fromBinary(geom2Param.buffer.nioBuffer(geom2Param.start, geom2Param.end - geom2Param.start));
+
+    int isDisjoint = geom1.disjoint(geom2) ? 1 : 0;
+
+    out.value = isDisjoint;
+  }
+}
diff --git a/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STDistance.java b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STDistance.java
new file mode 100644
index 0000000000..9415f3984e
--- /dev/null
+++ b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STDistance.java
@@ -0,0 +1,64 @@
+/*
+ * 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.fn.impl.gis;
+
+import javax.inject.Inject;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.Float8Holder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+
+import io.netty.buffer.DrillBuf;
+
+/*
+ * For geometry type Returns the 2D Cartesian distance between two geometries in projected units (based on spatial ref).
+ * For geography type defaults to return minimum geodesic distance between two geographies in meters
+ */
+@FunctionTemplate(name = "st_distance", scope = FunctionTemplate.FunctionScope.SIMPLE,
+  nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+public class STDistance implements DrillSimpleFunc {
+  @Param
+  VarBinaryHolder geom1Param;
+
+  @Param
+  VarBinaryHolder geom2Param;
+
+  @Output
+  Float8Holder out;
+
+  @Inject
+  DrillBuf buffer;
+
+  public void setup() {
+  }
+
+  public void eval() {
+    com.esri.core.geometry.ogc.OGCGeometry geom1;
+    com.esri.core.geometry.ogc.OGCGeometry geom2;
+
+    geom1 = com.esri.core.geometry.ogc.OGCGeometry
+        .fromBinary(geom1Param.buffer.nioBuffer(geom1Param.start, geom1Param.end - geom1Param.start));
+    geom2 = com.esri.core.geometry.ogc.OGCGeometry
+        .fromBinary(geom2Param.buffer.nioBuffer(geom2Param.start, geom2Param.end - geom2Param.start));
+
+    out.value = geom1.distance(geom2);
+  }
+}
diff --git a/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STEnvelope.java b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STEnvelope.java
new file mode 100644
index 0000000000..82862242fd
--- /dev/null
+++ b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STEnvelope.java
@@ -0,0 +1,69 @@
+/*
+ * 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.fn.impl.gis;
+
+import javax.inject.Inject;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+
+import io.netty.buffer.DrillBuf;
+
+/*
+ * Returns a geometry representing the double precision (float8) bounding box of the supplied geometry.
+ * The polygon is defined by the corner points of the bounding box ((MINX, MINY), (MINX, MAXY), (MAXX, MAXY), (MAXX, MINY), (MINX, MINY))
+ */
+@FunctionTemplate(name = "st_envelope", scope = FunctionTemplate.FunctionScope.SIMPLE,
+  nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+public class STEnvelope implements DrillSimpleFunc {
+  @Param
+  VarBinaryHolder geom1Param;
+
+  @Output
+  VarBinaryHolder out;
+
+  @Inject
+  DrillBuf buffer;
+
+  public void setup() {
+  }
+
+  public void eval() {
+    com.esri.core.geometry.ogc.OGCGeometry geom1;
+    geom1 = com.esri.core.geometry.ogc.OGCGeometry
+        .fromBinary(geom1Param.buffer.nioBuffer(geom1Param.start, geom1Param.end - geom1Param.start));
+
+    com.esri.core.geometry.ogc.OGCGeometry envelopeGeom;
+    if (geom1.geometryType().equals("Point")) {
+      envelopeGeom = geom1;
+    } else {
+      envelopeGeom = geom1.envelope();
+    }
+
+    java.nio.ByteBuffer envelopeGeomBytes = envelopeGeom.asBinary();
+
+    int outputSize = envelopeGeomBytes.remaining();
+    buffer = out.buffer = buffer.reallocIfNeeded(outputSize);
+    out.start = 0;
+    out.end = outputSize;
+    buffer.setBytes(0, envelopeGeomBytes);
+  }
+}
diff --git a/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STEquals.java b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STEquals.java
new file mode 100644
index 0000000000..b5e22d623e
--- /dev/null
+++ b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STEquals.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.expr.fn.impl.gis;
+
+import javax.inject.Inject;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.BitHolder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+
+import io.netty.buffer.DrillBuf;
+
+/*
+ * Returns true if the given geometries represent the same geometry. Directionality is ignored
+ */
+@FunctionTemplate(name = "st_equals", scope = FunctionTemplate.FunctionScope.SIMPLE,
+  nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+public class STEquals implements DrillSimpleFunc {
+  @Param
+  VarBinaryHolder geom1Param;
+
+  @Param
+  VarBinaryHolder geom2Param;
+
+  @Output
+  BitHolder out;
+
+  @Inject
+  DrillBuf buffer;
+
+  public void setup() {
+  }
+
+  public void eval() {
+    com.esri.core.geometry.ogc.OGCGeometry geom1;
+    com.esri.core.geometry.ogc.OGCGeometry geom2;
+
+    geom1 = com.esri.core.geometry.ogc.OGCGeometry
+        .fromBinary(geom1Param.buffer.nioBuffer(geom1Param.start, geom1Param.end - geom1Param.start));
+    geom2 = com.esri.core.geometry.ogc.OGCGeometry
+        .fromBinary(geom2Param.buffer.nioBuffer(geom2Param.start, geom2Param.end - geom2Param.start));
+
+    int equals = geom1.equals(geom2) ? 1 : 0;
+
+    out.value = equals;
+  }
+}
diff --git a/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STGeomFromText.java b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STGeomFromText.java
index 3a613e14bf..042046eb53 100644
--- a/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STGeomFromText.java
+++ b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STGeomFromText.java
@@ -45,12 +45,10 @@ public void setup() {
 
   public void eval() {
     String wktText = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(input.start, input.end,
-        input.buffer);
+            input.buffer);
 
     com.esri.core.geometry.ogc.OGCGeometry geom;
-
     geom = com.esri.core.geometry.ogc.OGCGeometry.fromText(wktText);
-
     java.nio.ByteBuffer pointBytes = geom.asBinary();
 
     int outputSize = pointBytes.remaining();
diff --git a/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STGeomFromTextSrid.java b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STGeomFromTextSrid.java
index 055eb948d0..9a7432e737 100644
--- a/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STGeomFromTextSrid.java
+++ b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STGeomFromTextSrid.java
@@ -64,5 +64,5 @@ public void eval() {
     out.start = 0;
     out.end = outputSize;
     buffer.setBytes(0, pointBytes);
-    }
+  }
 }
diff --git a/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STIntersects.java b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STIntersects.java
new file mode 100644
index 0000000000..9e152c5c6f
--- /dev/null
+++ b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STIntersects.java
@@ -0,0 +1,64 @@
+/*
+ * 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.fn.impl.gis;
+
+import javax.inject.Inject;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.BitHolder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+
+import io.netty.buffer.DrillBuf;
+
+/*
+ * Returns TRUE if the Geometries/Geography "spatially intersect in 2D" - (share any portion of space) and FALSE if they don't (they are Disjoint)
+ */
+@FunctionTemplate(name = "st_intersects", scope = FunctionTemplate.FunctionScope.SIMPLE,
+  nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+public class STIntersects implements DrillSimpleFunc {
+  @Param
+  VarBinaryHolder geom1Param;
+
+  @Param
+  VarBinaryHolder geom2Param;
+
+  @Output
+  BitHolder out;
+
+  @Inject
+  DrillBuf buffer;
+
+  public void setup() {
+  }
+
+  public void eval() {
+    com.esri.core.geometry.ogc.OGCGeometry geom1;
+    com.esri.core.geometry.ogc.OGCGeometry geom2;
+    geom1 = com.esri.core.geometry.ogc.OGCGeometry
+        .fromBinary(geom1Param.buffer.nioBuffer(geom1Param.start, geom1Param.end - geom1Param.start));
+    geom2 = com.esri.core.geometry.ogc.OGCGeometry
+        .fromBinary(geom2Param.buffer.nioBuffer(geom2Param.start, geom2Param.end - geom2Param.start));
+
+    int intersects = geom1.intersects(geom2) ? 1 : 0;
+
+    out.value = intersects;
+  }
+}
diff --git a/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STOverlaps.java b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STOverlaps.java
new file mode 100644
index 0000000000..7ae770d67e
--- /dev/null
+++ b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STOverlaps.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.expr.fn.impl.gis;
+
+import javax.inject.Inject;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.BitHolder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+
+import io.netty.buffer.DrillBuf;
+
+/*
+ * Returns TRUE if the Geometries share space, are of the same dimension, but are not completely contained by each other
+ */
+@FunctionTemplate(name = "st_overlaps", scope = FunctionTemplate.FunctionScope.SIMPLE,
+  nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+public class STOverlaps implements DrillSimpleFunc {
+  @Param
+  VarBinaryHolder geom1Param;
+
+  @Param
+  VarBinaryHolder geom2Param;
+
+  @Output
+  BitHolder out;
+
+  @Inject
+  DrillBuf buffer;
+
+  public void setup() {
+  }
+
+  public void eval() {
+    com.esri.core.geometry.ogc.OGCGeometry geom1;
+    com.esri.core.geometry.ogc.OGCGeometry geom2;
+
+    geom1 = com.esri.core.geometry.ogc.OGCGeometry
+        .fromBinary(geom1Param.buffer.nioBuffer(geom1Param.start, geom1Param.end - geom1Param.start));
+    geom2 = com.esri.core.geometry.ogc.OGCGeometry
+        .fromBinary(geom2Param.buffer.nioBuffer(geom2Param.start, geom2Param.end - geom2Param.start));
+
+    int overlaps = geom1.overlaps(geom2) ? 1 : 0;
+
+    out.value = overlaps;
+  }
+}
diff --git a/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STRelate.java b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STRelate.java
new file mode 100644
index 0000000000..9a1c64b227
--- /dev/null
+++ b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STRelate.java
@@ -0,0 +1,73 @@
+/*
+ * 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.fn.impl.gis;
+
+import javax.inject.Inject;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.BitHolder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.expr.holders.VarCharHolder;
+
+import io.netty.buffer.DrillBuf;
+
+/*
+ *  Returns true if this Geometry is spatially related to anotherGeometry, by testing for intersections between
+ *  the Interior, Boundary and Exterior of the two geometries as specified by the values in the intersectionMatrixPattern.
+ *  If no intersectionMatrixPattern is passed in, then returns the maximum intersectionMatrixPattern that relates the 2 geometries
+ */
+@FunctionTemplate(name = "st_relate", scope = FunctionTemplate.FunctionScope.SIMPLE,
+  nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+public class STRelate implements DrillSimpleFunc {
+  @Param
+  VarBinaryHolder geom1Param;
+
+  @Param
+  VarBinaryHolder geom2Param;
+
+  @Param
+  VarCharHolder matrixParam;
+
+  @Output
+  BitHolder out;
+
+  @Inject
+  DrillBuf buffer;
+
+  public void setup() {
+  }
+
+  public void eval() {
+    com.esri.core.geometry.ogc.OGCGeometry geom1;
+    com.esri.core.geometry.ogc.OGCGeometry geom2;
+    String matrix = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(matrixParam.start,
+        matrixParam.end, matrixParam.buffer);
+
+    geom1 = com.esri.core.geometry.ogc.OGCGeometry
+        .fromBinary(geom1Param.buffer.nioBuffer(geom1Param.start, geom1Param.end - geom1Param.start));
+    geom2 = com.esri.core.geometry.ogc.OGCGeometry
+        .fromBinary(geom2Param.buffer.nioBuffer(geom2Param.start, geom2Param.end - geom2Param.start));
+
+    int relates = geom1.relate(geom2, matrix) ? 1 : 0;
+
+    out.value = relates;
+  }
+}
diff --git a/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STTouches.java b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STTouches.java
new file mode 100644
index 0000000000..79ad0c73a2
--- /dev/null
+++ b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STTouches.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.expr.fn.impl.gis;
+
+import javax.inject.Inject;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.BitHolder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+
+import io.netty.buffer.DrillBuf;
+
+/*
+ * Returns TRUE if the geometries have at least one point in common, but their interiors do not intersect
+ */
+@FunctionTemplate(name = "st_touches", scope = FunctionTemplate.FunctionScope.SIMPLE,
+  nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+public class STTouches implements DrillSimpleFunc {
+  @Param
+  VarBinaryHolder geom1Param;
+
+  @Param
+  VarBinaryHolder geom2Param;
+
+  @Output
+  BitHolder out;
+
+  @Inject
+  DrillBuf buffer;
+
+  public void setup() {
+  }
+
+  public void eval() {
+    com.esri.core.geometry.ogc.OGCGeometry geom1;
+    com.esri.core.geometry.ogc.OGCGeometry geom2;
+
+    geom1 = com.esri.core.geometry.ogc.OGCGeometry
+        .fromBinary(geom1Param.buffer.nioBuffer(geom1Param.start, geom1Param.end - geom1Param.start));
+    geom2 = com.esri.core.geometry.ogc.OGCGeometry
+        .fromBinary(geom2Param.buffer.nioBuffer(geom2Param.start, geom2Param.end - geom2Param.start));
+
+    int touches = geom1.touches(geom2) ? 1 : 0;
+
+    out.value = touches;
+  }
+}
diff --git a/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STTransform.java b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STTransform.java
new file mode 100644
index 0000000000..c2788c18a5
--- /dev/null
+++ b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STTransform.java
@@ -0,0 +1,114 @@
+/*
+ * 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.fn.impl.gis;
+
+import javax.inject.Inject;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.expr.holders.NullableIntHolder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.osgeo.proj4j.CRSFactory;
+import org.osgeo.proj4j.CoordinateTransform;
+
+import io.netty.buffer.DrillBuf;
+
+/*
+ * Return a new geometry with its coordinates transformed to a different spatial reference
+ */
+@FunctionTemplate(name = "st_transform", scope = FunctionTemplate.FunctionScope.SIMPLE,
+  nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+public class STTransform implements DrillSimpleFunc {
+  @Param
+  VarBinaryHolder geom1Param;
+
+  @Param
+  NullableIntHolder sridSrcParam;
+
+  @Param
+  NullableIntHolder sridTgtParam;
+
+  @Workspace
+  CoordinateTransform transform;
+
+  @Workspace
+  CRSFactory crsFactory;
+
+  @Workspace
+  int sridTgt;
+
+  @Output
+  VarBinaryHolder out;
+
+  @Inject
+  DrillBuf buffer;
+
+  public void setup() {
+    int sridSrc = sridSrcParam.value;
+    sridTgt = sridTgtParam.value;
+
+    org.osgeo.proj4j.CoordinateReferenceSystem srcCrs =
+        new org.osgeo.proj4j.CRSFactory().createFromName("EPSG:" + sridSrc);
+
+    org.osgeo.proj4j.CoordinateReferenceSystem tgtCrs =
+        new org.osgeo.proj4j.CRSFactory().createFromName("EPSG:" + sridTgt);
+
+    transform = new org.osgeo.proj4j.BasicCoordinateTransform(srcCrs, tgtCrs);
+  }
+
+  public void eval() {
+    com.esri.core.geometry.ogc.OGCGeometry geomSrc = com.esri.core.geometry.ogc.OGCGeometry
+        .fromBinary(geom1Param.buffer.nioBuffer(geom1Param.start, geom1Param.end - geom1Param.start));
+
+    //int code = Integer.parseInt(transform.getTargetCRS().getName().substring(5, 9));//now sridTgt
+    org.osgeo.proj4j.ProjCoordinate result = new org.osgeo.proj4j.ProjCoordinate();
+    com.esri.core.geometry.SpatialReference sr = com.esri.core.geometry.SpatialReference.create(sridTgt);
+    java.nio.ByteBuffer geomBytes = null;
+
+    if (geomSrc != null && geomSrc.geometryType().equals("Point")) {
+      com.esri.core.geometry.ogc.OGCPoint pointGeom = (com.esri.core.geometry.ogc.OGCPoint) geomSrc;
+      result = transform.transform(new org.osgeo.proj4j.ProjCoordinate(pointGeom.X(), pointGeom.Y()), result);
+
+      geomBytes = new com.esri.core.geometry.ogc.OGCPoint(
+          new com.esri.core.geometry.Point(result.x, result.y), sr).asBinary();
+    } else {
+      com.esri.core.geometry.Geometry esriGeom = geomSrc.getEsriGeometry();
+      com.esri.core.geometry.MultiVertexGeometry vertexGeom =
+          com.esri.core.geometry.VertexGeomAccessor.getVertexGeometry(esriGeom);
+      for (int i = 0; i < vertexGeom.getPointCount(); i++) {
+        com.esri.core.geometry.Point point = vertexGeom.getPoint(i);
+        result = transform.transform(new org.osgeo.proj4j.ProjCoordinate(point.getX(), point.getY()), result);
+        point.setXY(result.x, result.y);
+        vertexGeom.setPoint(i, point);
+      }
+
+      com.esri.core.geometry.ogc.OGCGeometry tGeom =
+          com.esri.core.geometry.ogc.OGCGeometry.createFromEsriGeometry(esriGeom, sr);
+      geomBytes = tGeom.asBinary();
+    }
+
+    int outputSize = geomBytes.remaining();
+    buffer = out.buffer = buffer.reallocIfNeeded(outputSize);
+    out.start = 0;
+    out.end = outputSize;
+    buffer.setBytes(0, geomBytes);
+  }
+}
diff --git a/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STUnion.java b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STUnion.java
new file mode 100644
index 0000000000..907c6dadc2
--- /dev/null
+++ b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STUnion.java
@@ -0,0 +1,69 @@
+/*
+ * 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.fn.impl.gis;
+
+import javax.inject.Inject;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+
+import io.netty.buffer.DrillBuf;
+
+/*
+ * Returns a geometry that represents the point set union of the Geometries
+ */
+@FunctionTemplate(name = "st_union", scope = FunctionTemplate.FunctionScope.SIMPLE,
+  nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+public class STUnion implements DrillSimpleFunc {
+  @Param
+  VarBinaryHolder geom1Param;
+
+  @Param
+  VarBinaryHolder geom2Param;
+
+  @Output
+  VarBinaryHolder out;
+
+  @Inject
+  DrillBuf buffer;
+
+  public void setup() {
+  }
+
+  public void eval() {
+    com.esri.core.geometry.ogc.OGCGeometry geom1;
+    com.esri.core.geometry.ogc.OGCGeometry geom2;
+    geom1 = com.esri.core.geometry.ogc.OGCGeometry
+        .fromBinary(geom1Param.buffer.nioBuffer(geom1Param.start, geom1Param.end - geom1Param.start));
+    geom2 = com.esri.core.geometry.ogc.OGCGeometry
+        .fromBinary(geom2Param.buffer.nioBuffer(geom2Param.start, geom2Param.end - geom2Param.start));
+
+    com.esri.core.geometry.ogc.OGCGeometry unionGeom = geom1.union(geom2);
+
+    java.nio.ByteBuffer bufferedGeomBytes = unionGeom.asBinary();
+
+    int outputSize = bufferedGeomBytes.remaining();
+    buffer = out.buffer = buffer.reallocIfNeeded(outputSize);
+    out.start = 0;
+    out.end = outputSize;
+    buffer.setBytes(0, bufferedGeomBytes);
+  }
+}
diff --git a/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STUnionAggregate.java b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STUnionAggregate.java
new file mode 100644
index 0000000000..bcb761511d
--- /dev/null
+++ b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STUnionAggregate.java
@@ -0,0 +1,115 @@
+/*
+ * 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.fn.impl.gis;
+
+import javax.inject.Inject;
+
+import org.apache.drill.exec.expr.DrillAggFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.expr.holders.BigIntHolder;
+import org.apache.drill.exec.expr.holders.IntHolder;
+import org.apache.drill.exec.expr.holders.NullableVarBinaryHolder;
+import org.apache.drill.exec.expr.holders.ObjectHolder;
+import org.apache.drill.exec.expr.holders.UInt1Holder;
+
+import io.netty.buffer.DrillBuf;
+
+/*
+ * Returns a geometry that represents the point set union of the Geometries
+ */
+@FunctionTemplate(name = "st_unionaggregate", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+public class STUnionAggregate implements DrillAggFunc {
+  @Param NullableVarBinaryHolder in;
+  @Workspace ObjectHolder value;
+  @Workspace UInt1Holder init;
+  @Workspace BigIntHolder nonNullCount;
+  @Workspace IntHolder srid;
+  @Inject DrillBuf buf;
+  @Output NullableVarBinaryHolder out;
+
+  public void setup() {
+    init = new UInt1Holder();
+    nonNullCount = new BigIntHolder();
+    nonNullCount.value = 0;
+    init.value = 0;
+    value = new ObjectHolder();
+    value.obj = new java.util.ArrayList<com.esri.core.geometry.Geometry>();
+  }
+
+  @Override
+  public void add() {
+    sout: {
+      if (in.isSet == 0) {
+        // processing nullable input and the value is null, so don't do anything...
+        break sout;
+      }
+      nonNullCount.value = 1;
+      java.util.ArrayList<com.esri.core.geometry.Geometry> tmp = (java.util.ArrayList<com.esri.core.geometry.Geometry>) value.obj;
+
+      com.esri.core.geometry.ogc.OGCGeometry geom;
+      geom = com.esri.core.geometry.ogc.OGCGeometry
+          .fromBinary(in.buffer.nioBuffer(in.start, in.end - in.start));
+
+      tmp.add(geom.getEsriGeometry());
+
+      if(init.value == 0) {
+        init.value = 1;
+        srid.value = geom.SRID();
+      }
+    } // end of sout block
+  }
+
+  @Override
+  public void output() {
+    if (nonNullCount.value > 0) {
+      out.isSet = 1;
+
+      java.util.ArrayList<com.esri.core.geometry.Geometry> tmp = (java.util.ArrayList<com.esri.core.geometry.Geometry>) value.obj;
+
+      com.esri.core.geometry.SpatialReference spatialRef = null;
+      if (srid.value != 0){
+        spatialRef = com.esri.core.geometry.SpatialReference.create(4326);
+      }
+      com.esri.core.geometry.Geometry[] geomArr =
+          (com.esri.core.geometry.Geometry[]) tmp.toArray( new com.esri.core.geometry.Geometry[0] );
+      com.esri.core.geometry.Geometry geom = com.esri.core.geometry.GeometryEngine.union(geomArr, spatialRef);
+
+      com.esri.core.geometry.ogc.OGCGeometry unionGeom = com.esri.core.geometry.ogc.OGCGeometry.createFromEsriGeometry(geom, spatialRef);
+      java.nio.ByteBuffer unionGeomBytes = unionGeom.asBinary();
+
+      int outputSize = unionGeomBytes.remaining();
+      buf = out.buffer = buf.reallocIfNeeded(outputSize);
+      out.start = 0;
+      out.end = outputSize;
+      buf.setBytes(0, unionGeomBytes);
+    } else {
+      out.isSet = 0;
+    }
+  }
+
+  @Override
+  public void reset() {
+    value = new ObjectHolder();
+    value.obj = new java.util.ArrayList<com.esri.core.geometry.Geometry>();
+    init.value = 0;
+    nonNullCount.value = 0;
+  }
+}
diff --git a/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STXFunc.java b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STXFunc.java
new file mode 100644
index 0000000000..8986bbcc72
--- /dev/null
+++ b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STXFunc.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.expr.fn.impl.gis;
+
+import javax.inject.Inject;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.Float8Holder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+
+import io.netty.buffer.DrillBuf;
+
+/*
+ * Return the X coordinate of the point, or NaN if not available
+ */
+@FunctionTemplate(name = "st_x", scope = FunctionTemplate.FunctionScope.SIMPLE,
+  nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+public class STXFunc implements DrillSimpleFunc {
+  @Param
+  VarBinaryHolder geomParam;
+
+  @Output
+  Float8Holder out;
+
+  @Inject
+  DrillBuf buffer;
+
+  public void setup() {
+  }
+
+  public void eval() {
+
+    com.esri.core.geometry.ogc.OGCGeometry geom;
+
+    geom = com.esri.core.geometry.ogc.OGCGeometry
+        .fromBinary(geomParam.buffer.nioBuffer(geomParam.start, geomParam.end - geomParam.start));
+
+    if (geom != null && geom.geometryType().equals("Point")) {
+      out.value = ((com.esri.core.geometry.ogc.OGCPoint) geom).X();
+    } else {
+      out.value = Double.NaN;
+    }
+  }
+}
diff --git a/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STXMax.java b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STXMax.java
new file mode 100644
index 0000000000..ee8afa0bc9
--- /dev/null
+++ b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STXMax.java
@@ -0,0 +1,63 @@
+/*
+ * 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.fn.impl.gis;
+
+import javax.inject.Inject;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.Float8Holder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+
+import io.netty.buffer.DrillBuf;
+
+/*
+ * Returns X maxima of a bounding box 2d or 3d or a geometry
+ */
+@FunctionTemplate(name = "st_xmax", scope = FunctionTemplate.FunctionScope.SIMPLE,
+  nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+public class STXMax implements DrillSimpleFunc {
+  @Param
+  VarBinaryHolder geom1Param;
+
+  @Output
+  Float8Holder out;
+
+  @Inject
+  DrillBuf buffer;
+
+  public void setup() {
+  }
+
+  public void eval() {
+    com.esri.core.geometry.ogc.OGCGeometry geom1;
+    geom1 = com.esri.core.geometry.ogc.OGCGeometry
+        .fromBinary(geom1Param.buffer.nioBuffer(geom1Param.start, geom1Param.end - geom1Param.start));
+
+    com.esri.core.geometry.ogc.OGCGeometry envelopeGeom;
+    if (geom1.geometryType().equals("Point")) {
+      out.value = ((com.esri.core.geometry.ogc.OGCPoint) geom1).X();
+    } else {
+      com.esri.core.geometry.Envelope envelope = new com.esri.core.geometry.Envelope();
+      geom1.getEsriGeometry().queryEnvelope(envelope);
+      out.value = envelope.getXMax();
+    }
+  }
+}
diff --git a/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STXMin.java b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STXMin.java
new file mode 100644
index 0000000000..d527e6e49f
--- /dev/null
+++ b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STXMin.java
@@ -0,0 +1,63 @@
+/*
+ * 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.fn.impl.gis;
+
+import javax.inject.Inject;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.Float8Holder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+
+import io.netty.buffer.DrillBuf;
+
+/*
+ * Returns X minima of a bounding box 2d or 3d or a geometry
+ */
+@FunctionTemplate(name = "st_xmin", scope = FunctionTemplate.FunctionScope.SIMPLE,
+  nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+public class STXMin implements DrillSimpleFunc {
+  @Param
+  VarBinaryHolder geom1Param;
+
+  @Output
+  Float8Holder out;
+
+  @Inject
+  DrillBuf buffer;
+
+  public void setup() {
+  }
+
+  public void eval() {
+    com.esri.core.geometry.ogc.OGCGeometry geom1;
+    geom1 = com.esri.core.geometry.ogc.OGCGeometry
+        .fromBinary(geom1Param.buffer.nioBuffer(geom1Param.start, geom1Param.end - geom1Param.start));
+
+    com.esri.core.geometry.ogc.OGCGeometry envelopeGeom;
+    if (geom1.geometryType().equals("Point")) {
+      out.value = ((com.esri.core.geometry.ogc.OGCPoint) geom1).X();
+    } else {
+      com.esri.core.geometry.Envelope envelope = new com.esri.core.geometry.Envelope();
+      geom1.getEsriGeometry().queryEnvelope(envelope);
+      out.value = envelope.getXMin();
+    }
+  }
+}
diff --git a/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STYFunc.java b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STYFunc.java
new file mode 100644
index 0000000000..ce1771addd
--- /dev/null
+++ b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STYFunc.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.expr.fn.impl.gis;
+
+import javax.inject.Inject;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.Float8Holder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+
+import io.netty.buffer.DrillBuf;
+
+/*
+ * Return the Y coordinate of the point, or NaN if not available
+ */
+@FunctionTemplate(name = "st_y", scope = FunctionTemplate.FunctionScope.SIMPLE,
+  nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+public class STYFunc implements DrillSimpleFunc {
+  @Param
+  VarBinaryHolder geomParam;
+
+  @Output
+  Float8Holder out;
+
+  @Inject
+  DrillBuf buffer;
+
+  public void setup() {
+  }
+
+  public void eval() {
+
+    com.esri.core.geometry.ogc.OGCGeometry geom;
+
+    geom = com.esri.core.geometry.ogc.OGCGeometry
+        .fromBinary(geomParam.buffer.nioBuffer(geomParam.start, geomParam.end - geomParam.start));
+
+    if (geom != null && geom.geometryType().equals("Point")) {
+      out.value = ((com.esri.core.geometry.ogc.OGCPoint) geom).Y();
+    } else {
+      out.value = Double.NaN;
+    }
+  }
+}
diff --git a/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STYMax.java b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STYMax.java
new file mode 100644
index 0000000000..4101e79f2d
--- /dev/null
+++ b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STYMax.java
@@ -0,0 +1,63 @@
+/*
+ * 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.fn.impl.gis;
+
+import javax.inject.Inject;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.Float8Holder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+
+import io.netty.buffer.DrillBuf;
+
+/*
+ * Returns Y maxima of a bounding box 2d or 3d or a geometry
+ */
+@FunctionTemplate(name = "st_ymax", scope = FunctionTemplate.FunctionScope.SIMPLE,
+  nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+public class STYMax implements DrillSimpleFunc {
+  @Param
+  VarBinaryHolder geom1Param;
+
+  @Output
+  Float8Holder out;
+
+  @Inject
+  DrillBuf buffer;
+
+  public void setup() {
+  }
+
+  public void eval() {
+    com.esri.core.geometry.ogc.OGCGeometry geom1;
+    geom1 = com.esri.core.geometry.ogc.OGCGeometry
+        .fromBinary(geom1Param.buffer.nioBuffer(geom1Param.start, geom1Param.end - geom1Param.start));
+
+    com.esri.core.geometry.ogc.OGCGeometry envelopeGeom;
+    if (geom1.geometryType().equals("Point")) {
+      out.value = ((com.esri.core.geometry.ogc.OGCPoint) geom1).Y();
+    } else {
+      com.esri.core.geometry.Envelope envelope = new com.esri.core.geometry.Envelope();
+      geom1.getEsriGeometry().queryEnvelope(envelope);
+      out.value = envelope.getYMax();
+    }
+  }
+}
diff --git a/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STYMin.java b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STYMin.java
new file mode 100644
index 0000000000..85fc7e45c1
--- /dev/null
+++ b/contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STYMin.java
@@ -0,0 +1,63 @@
+/*
+ * 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.fn.impl.gis;
+
+import javax.inject.Inject;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.Float8Holder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+
+import io.netty.buffer.DrillBuf;
+
+/*
+ * Returns Y minima of a bounding box 2d or 3d or a geometry
+ */
+@FunctionTemplate(name = "st_ymin", scope = FunctionTemplate.FunctionScope.SIMPLE,
+  nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+public class STYMin implements DrillSimpleFunc {
+  @Param
+  VarBinaryHolder geom1Param;
+
+  @Output
+  Float8Holder out;
+
+  @Inject
+  DrillBuf buffer;
+
+  public void setup() {
+  }
+
+  public void eval() {
+    com.esri.core.geometry.ogc.OGCGeometry geom1;
+    geom1 = com.esri.core.geometry.ogc.OGCGeometry
+        .fromBinary(geom1Param.buffer.nioBuffer(geom1Param.start, geom1Param.end - geom1Param.start));
+
+    com.esri.core.geometry.ogc.OGCGeometry envelopeGeom;
+    if (geom1.geometryType().equals("Point")) {
+      out.value = ((com.esri.core.geometry.ogc.OGCPoint) geom1).Y();
+    } else {
+      com.esri.core.geometry.Envelope envelope = new com.esri.core.geometry.Envelope();
+      geom1.getEsriGeometry().queryEnvelope(envelope);
+      out.value = envelope.getYMin();
+    }
+  }
+}
diff --git a/contrib/gis/src/test/java/org/apache/drill/exec/expr/fn/impl/gis/TestGeometryFunctions.java b/contrib/gis/src/test/java/org/apache/drill/exec/expr/fn/impl/gis/TestGeometryFunctions.java
index 8e5ead3d1e..379de5c47e 100644
--- a/contrib/gis/src/test/java/org/apache/drill/exec/expr/fn/impl/gis/TestGeometryFunctions.java
+++ b/contrib/gis/src/test/java/org/apache/drill/exec/expr/fn/impl/gis/TestGeometryFunctions.java
@@ -29,7 +29,6 @@
 
   @Test
   public void testGeometryFromTextCreation() throws Exception {
-
     testBuilder()
     .sqlQuery("select ST_AsText(ST_GeomFromText('" + wktPoint + "')) "
         + "from cp.`sample-data/CA-cities.csv` limit 1")
@@ -41,7 +40,6 @@ public void testGeometryFromTextCreation() throws Exception {
 
   @Test
   public void testGeometryPointCreation() throws Exception {
-
     testBuilder()
       .sqlQuery("select ST_AsText(ST_Point(-121.895, 37.339)) "
           + "from cp.`sample-data/CA-cities.csv` limit 1")
@@ -75,6 +73,30 @@ public void testJSONFromTextCreation() throws Exception {
       .run();
   }
 
+  @Test
+  public void testNullWkt() throws Exception {
+    testBuilder()
+      .sqlQuery("select ST_AsText(ST_GeomFromText(columns[4])) " +
+              "from cp.`/sample-data/CA-cities-with-nulls.csv` limit 1")
+      .ordered()
+      .baselineColumns("EXPR$0")
+      .baselineValues(new Object[]{null})
+      .build()
+      .run();
+  }
+
+  @Test
+  public void testNullGeoJSON() throws Exception {
+    testBuilder()
+            .sqlQuery("select ST_AsGeoJson(ST_GeomFromText(columns[4])) " +
+                    "from cp.`/sample-data/CA-cities-with-nulls.csv` limit 1")
+            .ordered()
+            .baselineColumns("EXPR$0")
+            .baselineValues(new Object[]{null})
+            .build()
+            .run();
+  }
+
   @Test
   public void testGeoJSONCreationFromPoint() throws Exception {
     testBuilder()
@@ -86,7 +108,6 @@ public void testGeoJSONCreationFromPoint() throws Exception {
       .build()
       .run();
   }
-
   @Test
   public void testGeoJSONCreationFromGeom() throws Exception {
     testBuilder()
@@ -101,7 +122,6 @@ public void testGeoJSONCreationFromGeom() throws Exception {
 
   @Test
   public void testSTWithinQuery() throws Exception {
-
     testBuilder()
       .sqlQuery("select ST_Within(ST_Point(columns[4], columns[3]),"
           + "ST_GeomFromText('POLYGON((-121.95 37.28, -121.94 37.35, -121.84 37.35, -121.84 37.28, -121.95 37.28))')"
@@ -123,4 +143,204 @@ public void testSTWithinQuery() throws Exception {
     .build()
     .run();
   }
+
+  @Test
+  public void testSTXQuery() throws Exception {
+    testBuilder()
+      .sqlQuery("select ST_X(ST_Point(-121.895, 37.339)) "
+          + "from cp.`/sample-data/CA-cities.csv` limit 1")
+      .ordered().baselineColumns("EXPR$0")
+      .baselineValues(-121.895)
+      .build()
+      .run();
+  }
+
+  @Test
+  public void testSTYQuery() throws Exception {
+    testBuilder()
+      .sqlQuery("select ST_Y(ST_Point(-121.895, 37.339)) "
+          + "from cp.`/sample-data/CA-cities.csv` limit 1")
+      .ordered().baselineColumns("EXPR$0")
+      .baselineValues(37.339)
+      .build()
+      .run();
+  }
+
+  @Test
+  public void testIntersectQuery() throws Exception {
+    testBuilder()
+      .sqlQuery("SELECT ST_Intersects(ST_GeomFromText('POINT(0 0)'), ST_GeomFromText('LINESTRING(2 0,0 2)')) "
+          + "from (VALUES(1))")
+      .ordered().baselineColumns("EXPR$0")
+      .baselineValues(false)
+      .build()
+      .run();
+
+    testBuilder()
+    .sqlQuery("SELECT ST_Intersects(ST_GeomFromText('POINT(0 0)'), ST_GeomFromText('LINESTRING(0 0,0 2)')) "
+        + "from (VALUES(1))")
+    .ordered().baselineColumns("EXPR$0")
+    .baselineValues(true)
+    .build()
+    .run();
+  }
+
+  @Test
+  public void testRelateQuery() throws Exception {
+    testBuilder()
+      .sqlQuery("SELECT ST_Relate(ST_GeomFromText('POINT(1 2)'), ST_Buffer(ST_GeomFromText('POINT(1 2)'),2), '0FFFFF212') "
+          + "from (VALUES(1))")
+      .ordered().baselineColumns("EXPR$0")
+      .baselineValues(true)
+      .build()
+      .run();
+
+    testBuilder()
+    .sqlQuery("SELECT ST_Relate(ST_GeomFromText('POINT(1 2)'), ST_Buffer(ST_GeomFromText('POINT(1 2)'),2), '*FF*FF212') "
+        + "from (VALUES(1))")
+    .ordered().baselineColumns("EXPR$0")
+    .baselineValues(true)
+    .build()
+    .run();
+
+    testBuilder()
+    .sqlQuery("SELECT ST_Relate(ST_GeomFromText('POINT(0 0)'), ST_Buffer(ST_GeomFromText('POINT(1 2)'),2), '*FF*FF212') "
+        + "from (VALUES(1))")
+    .ordered().baselineColumns("EXPR$0")
+    .baselineValues(false)
+    .build()
+    .run();
+  }
+
+  @Test
+  public void testTouchesQuery() throws Exception {
+    testBuilder()
+      .sqlQuery("SELECT ST_Touches(ST_GeomFromText('LINESTRING(0 0, 1 1, 0 2)'), ST_GeomFromText('POINT(1 1)')) "
+          + "from (VALUES(1))")
+      .ordered().baselineColumns("EXPR$0")
+      .baselineValues(false)
+      .build()
+      .run();
+
+    testBuilder()
+    .sqlQuery("SELECT ST_Touches(ST_GeomFromText('LINESTRING(0 0, 1 1, 0 2)'), ST_GeomFromText('POINT(0 2)')) "
+        + "from (VALUES(1))")
+    .ordered().baselineColumns("EXPR$0")
+    .baselineValues(true)
+    .build()
+    .run();
+  }
+
+  @Test
+  public void testEqualsQuery() throws Exception {
+    testBuilder()
+      .sqlQuery("SELECT ST_Equals(ST_GeomFromText('LINESTRING(0 0, 10 10)'), "
+                + "ST_GeomFromText('LINESTRING(0 0, 5 5, 10 10)')) from (VALUES(1))")
+      .ordered().baselineColumns("EXPR$0")
+      .baselineValues(true)
+      .build()
+      .run();
+  }
+
+  @Test
+  public void testContainsQuery() throws Exception {
+    testBuilder()
+      .sqlQuery("SELECT ST_Contains(smallc, bigc) As smallcontainsbig, "
+                     + "ST_Contains(bigc,smallc) As bigcontainssmall, "
+                     + "ST_Contains(bigc, ST_Union(smallc, bigc)) as bigcontainsunion, "
+                     + "ST_Equals(bigc, ST_Union(smallc, bigc)) as bigisunion "
+                + "FROM (SELECT ST_Buffer(ST_GeomFromText('POINT(1 2)'), 10) As smallc, "
+                       + "ST_Buffer(ST_GeomFromText('POINT(1 2)'), 20) As bigc from (VALUES(1)) ) As foo")
+      .ordered().baselineColumns("smallcontainsbig", "bigcontainssmall", "bigcontainsunion", "bigisunion")
+      .baselineValues(false, true, true, true)
+      .build()
+      .run();
+  }
+
+  @Test
+  public void testOverlapsCrossesIntersectsContainsQuery() throws Exception {
+    testBuilder()
+      .sqlQuery("SELECT ST_Overlaps(a,b) As a_overlap_b, "
+                  + "ST_Crosses(a,b) As a_crosses_b, "
+                  + "ST_Intersects(a, b) As a_intersects_b, "
+                  + "ST_Contains(b,a) As b_contains_a "
+                + "FROM (SELECT ST_GeomFromText('POINT(1 0.5)') As a, ST_GeomFromText('LINESTRING(1 0, 1 1, 3 5)')  As b "
+                  + "from (VALUES(1)) ) As foo")
+      .ordered().baselineColumns("a_overlap_b", "a_crosses_b", "a_intersects_b", "b_contains_a")
+      .baselineValues(false, false, true, true)
+      .build()
+      .run();
+  }
+
+  @Test
+  public void testDisjointQuery() throws Exception {
+    testBuilder()
+      .sqlQuery("SELECT ST_Disjoint(ST_GeomFromText('POINT(0 0)'), ST_GeomFromText('LINESTRING( 2 0, 0 2 )')) "
+                + "from (VALUES(1))")
+      .ordered().baselineColumns("EXPR$0")
+      .baselineValues(true)
+      .build()
+      .run();
+
+    testBuilder()
+    .sqlQuery("SELECT ST_Disjoint(ST_GeomFromText('POINT(0 0)'), ST_GeomFromText('LINESTRING( 0 0, 0 2 )')) "
+              + "from (VALUES(1))")
+    .ordered().baselineColumns("EXPR$0")
+    .baselineValues(false)
+    .build()
+    .run();
+  }
+
+  @Test
+  public void testTransfromQuery() throws Exception {
+    double targetX = -71.1776848522251;
+    double targetY = 42.3902896512902;
+
+    testBuilder()
+      .sqlQuery("SELECT round(st_x(st_transform(st_geomfromtext('POINT (743238 2967416)'), 2249, 4326)), 13),"
+          + " round(st_y(st_transform(st_geomfromtext('POINT (743238 2967416)'), 2249, 4326)), 13) from (VALUES(1))")
+      .ordered().baselineColumns("EXPR$0", "EXPR$1")
+      .baselineValues(targetX, targetY)
+      .build()
+      .run();
+  }
+
+  @Test
+  public void testUnionAggregateQuery() throws Exception {
+    String targetAll = "MULTIPOLYGON (((0 -1, 1 -1, 1 0, 1 1, 0 1, 0 0, 0 -1)), "
+                        + "((10 9, 11 9, 11 10, 11 11, 10 11, 10 10, 10 9)))";
+    String targetFirstGroup = "POLYGON ((0 -1, 1 -1, 1 0, 1 1, 0 1, 0 0, 0 -1))";
+    String targetSecondGroup = "POLYGON ((10 9, 11 9, 11 10, 11 11, 10 11, 10 10, 10 9))";
+
+    testBuilder()
+      .sqlQuery("select ST_AsText(ST_UnionAggregate(ST_GeomFromText(columns[1]))) from cp.`sample-data/polygons.tsv`")
+      .ordered().baselineColumns("EXPR$0")
+      .baselineValues(targetAll)
+      .build()
+      .run();
+
+    testBuilder()
+      .sqlQuery("select columns[0], ST_AsText(ST_UnionAggregate(ST_GeomFromText(columns[1])))"
+          + " from cp.`sample-data/polygons.tsv` group by columns[0] having columns[0] = '1'")
+      .ordered().baselineColumns("EXPR$0", "EXPR$1")
+      .baselineValues("1", targetFirstGroup)
+      .build()
+      .run();
+
+    testBuilder()
+      .sqlQuery("select columns[0], ST_AsText(ST_UnionAggregate(ST_GeomFromText(columns[1])))"
+          + " from cp.`sample-data/polygons.tsv` group by columns[0] having columns[0] = '2'")
+      .ordered().baselineColumns("EXPR$0", "EXPR$1")
+      .baselineValues("2", targetSecondGroup)
+      .build()
+      .run();
+
+    testBuilder()
+    .sqlQuery("select count(*) from (select columns[0], ST_AsText(ST_UnionAggregate(ST_GeomFromText(columns[1])))"
+        + " from cp.`sample-data/polygons.tsv` group by columns[0])")
+    .ordered().baselineColumns("EXPR$0")
+    .baselineValues(3L)
+    .build()
+    .run();
+  }
 }
diff --git a/contrib/native/client/CMakeLists.txt b/contrib/native/client/CMakeLists.txt
index 3f6c44b609..f434c41021 100644
--- a/contrib/native/client/CMakeLists.txt
+++ b/contrib/native/client/CMakeLists.txt
@@ -101,13 +101,14 @@ find_package(Boost 1.53.0 REQUIRED COMPONENTS regex system date_time chrono thre
 include_directories(${Boost_INCLUDE_DIRS})
 
 
-if(CMAKE_COMPILER_IS_GNUCXX)
-    set(CMAKE_EXE_LINKER_FLAGS "-lrt -lpthread")
-    set(CMAKE_CXX_FLAGS "-fPIC")
+if(CMAKE_COMPILER_IS_GNUCXX OR CMAKE_COMPILER_IS_GNUCC)
+    set(CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -lrt -lpthread")
+    set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fPIC")
+    set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fPIC")
 endif()
 
 if(MSVC)
-    set(CMAKE_CXX_FLAGS "/EHsc")
+    set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} /EHsc")
 endif()
 
 if(MSVC)
diff --git a/contrib/native/client/src/clientlib/CMakeLists.txt b/contrib/native/client/src/clientlib/CMakeLists.txt
index 7b9ecc3c0f..90cfe8b6fa 100644
--- a/contrib/native/client/src/clientlib/CMakeLists.txt
+++ b/contrib/native/client/src/clientlib/CMakeLists.txt
@@ -16,6 +16,13 @@
 # limitations under the License.
 #
 
+if(MSVC)
+    configure_file(
+      ${CMAKE_CURRENT_SOURCE_DIR}/version.rc.in
+      ${CMAKE_CURRENT_BINARY_DIR}/version.rc
+      @ONLY)
+endif()
+
 # Drill Client library
 
 set (CLIENTLIB_SRC_FILES
@@ -50,9 +57,10 @@ set_property(
     )
 
 if(MSVC)
-    set(CMAKE_CXX_FLAGS "/EHsc")
     add_definitions(-DDRILL_CLIENT_EXPORTS -D_SCL_SECURE_NO_WARNINGS)
+    add_library(drillClient SHARED ${CLIENTLIB_SRC_FILES} ${CMAKE_CURRENT_BINARY_DIR}/version.rc)
+else()
+    add_library(drillClient SHARED ${CLIENTLIB_SRC_FILES})
 endif()
 
-add_library(drillClient SHARED ${CLIENTLIB_SRC_FILES} )
 target_link_libraries(drillClient ${Boost_LIBRARIES} ${PROTOBUF_LIBRARY} ${Zookeeper_LIBRARIES} ${SASL_LIBRARIES} ${OPENSSL_LIBRARIES} protomsgs y2038)
diff --git a/contrib/native/client/src/clientlib/env.h.in b/contrib/native/client/src/clientlib/env.h.in
index 380746a3fb..794b6282b6 100644
--- a/contrib/native/client/src/clientlib/env.h.in
+++ b/contrib/native/client/src/clientlib/env.h.in
@@ -19,7 +19,7 @@
 #define ENV_H
 
 #define DRILL_NAME              "Apache Drill"
-#define DRILL_CONNECTOR_NAME    "Apache Drill C++ client"
+#define DRILL_CONNECTOR_NAME    "Apache Drill C++ Client"
 #define DRILL_VERSION_STRING    "@PROJECT_VERSION@"
 
 #define DRILL_VERSION_MAJOR @PROJECT_VERSION_MAJOR@
@@ -29,6 +29,11 @@
 #define GIT_SHA_PROP  @GIT_SHA_PROP@
 #define GIT_COMMIT_PROP @GIT_COMMIT_PROP@
 
+#define DRILL_LEGALCOPYRIGHT_STR    "Copyright (c) 2013-2017 The Apache Software Foundation\0"
+#define DRILL_PRODUCTNAME_STR       DRILL_CONNECTOR_NAME "\0"
+#define DRILL_PRODUCTVERSION_STR    DRILL_VERSION_STRING ".0\0"
+#define DRILL_INTERNALNAME_STR      "drillClient.dll\0"
+
 #endif
 
 
diff --git a/contrib/native/client/src/clientlib/version.rc.in b/contrib/native/client/src/clientlib/version.rc.in
new file mode 100644
index 0000000000..c013261cc6
--- /dev/null
+++ b/contrib/native/client/src/clientlib/version.rc.in
@@ -0,0 +1,68 @@
+/*
+ * 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.
+ */
+
+ #include "env.h"
+ 
+#define APSTUDIO_READONLY_SYMBOLS
+/////////////////////////////////////////////////////////////////////////////
+//
+// Generated resource.
+//
+#include "afxres.h"
+
+/////////////////////////////////////////////////////////////////////////////
+#undef APSTUDIO_READONLY_SYMBOLS
+
+/////////////////////////////////////////////////////////////////////////////
+// English (US) resources.
+LANGUAGE LANG_ENGLISH, SUBLANG_ENGLISH_US
+#pragma code_page(1252)
+
+VS_VERSION_INFO VERSIONINFO
+ FILEVERSION @PROJECT_VERSION_MAJOR@,@PROJECT_VERSION_MINOR@,@PROJECT_VERSION_PATCH@,0
+ PRODUCTVERSION @PROJECT_VERSION_MAJOR@,@PROJECT_VERSION_MINOR@,@PROJECT_VERSION_PATCH@,0
+ FILEFLAGSMASK 0x3fL 
+#ifdef _DEBUG
+ FILEFLAGS 0x1L
+#else
+ FILEFLAGS 0x0L
+#endif
+ FILEOS 0x4L
+ FILETYPE 0x0L
+ FILESUBTYPE 0x0L
+BEGIN
+    BLOCK "StringFileInfo"
+    BEGIN
+        BLOCK "040904b0"
+        BEGIN
+            VALUE "CompanyName", "\0"
+            VALUE "FileDescription", DRILL_PRODUCTNAME_STR
+            VALUE "FileVersion", DRILL_PRODUCTVERSION_STR
+            VALUE "LegalCopyright", DRILL_LEGALCOPYRIGHT_STR
+            VALUE "ProductName", DRILL_PRODUCTNAME_STR
+            VALUE "ProductVersion", DRILL_PRODUCTVERSION_STR
+            VALUE "InternalName", DRILL_INTERNALNAME_STR
+            VALUE "OriginalFilename", DRILL_INTERNALNAME_STR
+            VALUE "LegalTrademarks", "\0"
+        END
+    END
+    BLOCK "VarFileInfo"
+    BEGIN
+        VALUE "Translation", 0x409, 1200
+    END
+END
\ No newline at end of file
diff --git a/contrib/native/client/src/clientlib/y2038/CMakeLists.txt b/contrib/native/client/src/clientlib/y2038/CMakeLists.txt
index 1cb4d72223..bfc3fb4e62 100644
--- a/contrib/native/client/src/clientlib/y2038/CMakeLists.txt
+++ b/contrib/native/client/src/clientlib/y2038/CMakeLists.txt
@@ -18,14 +18,6 @@
 
 # Y2038 library
 
-if(CMAKE_COMPILER_IS_GNUCXX OR CMAKE_COMPILER_IS_GNUCC)
-    set(CMAKE_CXX_FLAGS "-fPIC")
-endif()
-
-if(CMAKE_COMPILER_IS_GNUCC)
-    set(CMAKE_C_FLAGS "-fPIC")
-endif()
-
 set (Y2038_SRC_FILES
     ${CMAKE_CURRENT_SOURCE_DIR}/time64.c
     )
diff --git a/contrib/native/client/src/include/drill/drillClient.hpp b/contrib/native/client/src/include/drill/drillClient.hpp
index a09e6661b0..7eabb50915 100644
--- a/contrib/native/client/src/include/drill/drillClient.hpp
+++ b/contrib/native/client/src/include/drill/drillClient.hpp
@@ -201,15 +201,15 @@ namespace meta {
     _DL_INTERVAL_MINUTE_TO_SECOND = 1 << 16L
   };
 
-  template inline _DateTimeLiteralSupport operator&(_DateTimeLiteralSupport __a, _DateTimeLiteralSupport __b);
-  template inline _DateTimeLiteralSupport operator|(_DateTimeLiteralSupport __a, _DateTimeLiteralSupport __b);
-  template inline _DateTimeLiteralSupport operator^(_DateTimeLiteralSupport __a, _DateTimeLiteralSupport __b);
+  template _DateTimeLiteralSupport operator&(_DateTimeLiteralSupport __a, _DateTimeLiteralSupport __b);
+  template _DateTimeLiteralSupport operator|(_DateTimeLiteralSupport __a, _DateTimeLiteralSupport __b);
+  template _DateTimeLiteralSupport operator^(_DateTimeLiteralSupport __a, _DateTimeLiteralSupport __b);
 
-  template inline _DateTimeLiteralSupport& operator&=(_DateTimeLiteralSupport& __a, _DateTimeLiteralSupport __b);
-  template inline _DateTimeLiteralSupport& operator|=(_DateTimeLiteralSupport& __a, _DateTimeLiteralSupport __b);
-  template inline _DateTimeLiteralSupport& operator^=(_DateTimeLiteralSupport& __a, _DateTimeLiteralSupport __b);
+  template _DateTimeLiteralSupport& operator&=(_DateTimeLiteralSupport& __a, _DateTimeLiteralSupport __b);
+  template _DateTimeLiteralSupport& operator|=(_DateTimeLiteralSupport& __a, _DateTimeLiteralSupport __b);
+  template _DateTimeLiteralSupport& operator^=(_DateTimeLiteralSupport& __a, _DateTimeLiteralSupport __b);
 
-  template inline _DateTimeLiteralSupport operator~(_DateTimeLiteralSupport __a);
+  template _DateTimeLiteralSupport operator~(_DateTimeLiteralSupport __a);
 
   /**
    * Date time literal support flags
@@ -259,15 +259,15 @@ namespace meta {
       _C_GROUPBY    = 1 << 1L
   };
 
-  template inline _CollateSupport operator&(_CollateSupport __a, _CollateSupport __b);
-  template inline _CollateSupport operator|(_CollateSupport __a, _CollateSupport __b);
-  template inline _CollateSupport operator^(_CollateSupport __a, _CollateSupport __b);
+  template _CollateSupport operator&(_CollateSupport __a, _CollateSupport __b);
+  template _CollateSupport operator|(_CollateSupport __a, _CollateSupport __b);
+  template _CollateSupport operator^(_CollateSupport __a, _CollateSupport __b);
 
-  template inline _CollateSupport& operator&=(_CollateSupport& __a, _CollateSupport __b);
-  template inline _CollateSupport& operator|=(_CollateSupport& __a, _CollateSupport __b);
-  template inline _CollateSupport& operator^=(_CollateSupport& __a, _CollateSupport __b);
+  template _CollateSupport& operator&=(_CollateSupport& __a, _CollateSupport __b);
+  template _CollateSupport& operator|=(_CollateSupport& __a, _CollateSupport __b);
+  template _CollateSupport& operator^=(_CollateSupport& __a, _CollateSupport __b);
 
-  template inline _CollateSupport operator~(_CollateSupport __a);
+  template _CollateSupport operator~(_CollateSupport __a);
 
 
   /**
@@ -339,15 +339,15 @@ namespace meta {
       _OJ_ALL_COMPARISON_OPS    = 1 << 7L //!< _OJ_ALL_COMPARISON_OPS
   };
 
-  template inline _OuterJoinSupport operator&(_OuterJoinSupport __a, _OuterJoinSupport __b);
-  template inline _OuterJoinSupport operator|(_OuterJoinSupport __a, _OuterJoinSupport __b);
-  template inline _OuterJoinSupport operator^(_OuterJoinSupport __a, _OuterJoinSupport __b);
+  template _OuterJoinSupport operator&(_OuterJoinSupport __a, _OuterJoinSupport __b);
+  template _OuterJoinSupport operator|(_OuterJoinSupport __a, _OuterJoinSupport __b);
+  template _OuterJoinSupport operator^(_OuterJoinSupport __a, _OuterJoinSupport __b);
 
-  template inline _OuterJoinSupport& operator&=(_OuterJoinSupport& __a, _OuterJoinSupport __b);
-  template inline _OuterJoinSupport& operator|=(_OuterJoinSupport& __a, _OuterJoinSupport __b);
-  template inline _OuterJoinSupport& operator^=(_OuterJoinSupport& __a, _OuterJoinSupport __b);
+  template _OuterJoinSupport& operator&=(_OuterJoinSupport& __a, _OuterJoinSupport __b);
+  template _OuterJoinSupport& operator|=(_OuterJoinSupport& __a, _OuterJoinSupport __b);
+  template _OuterJoinSupport& operator^=(_OuterJoinSupport& __a, _OuterJoinSupport __b);
 
-  template inline _OuterJoinSupport operator~(_OuterJoinSupport __a);
+  template _OuterJoinSupport operator~(_OuterJoinSupport __a);
 
   /**
    * Outer join support flags
@@ -404,15 +404,15 @@ namespace meta {
       _SQ_IN_QUANTIFIED = 1 << 5L
   };
 
-  template inline _SubQuerySupport operator&(_SubQuerySupport __a, _SubQuerySupport __b);
-  template inline _SubQuerySupport operator|(_SubQuerySupport __a, _SubQuerySupport __b);
-  template inline _SubQuerySupport operator^(_SubQuerySupport __a, _SubQuerySupport __b);
+  template _SubQuerySupport operator&(_SubQuerySupport __a, _SubQuerySupport __b);
+  template _SubQuerySupport operator|(_SubQuerySupport __a, _SubQuerySupport __b);
+  template _SubQuerySupport operator^(_SubQuerySupport __a, _SubQuerySupport __b);
 
-  template inline _SubQuerySupport& operator&=(_SubQuerySupport& __a, _SubQuerySupport __b);
-  template inline _SubQuerySupport& operator|=(_SubQuerySupport& __a, _SubQuerySupport __b);
-  template inline _SubQuerySupport& operator^=(_SubQuerySupport& __a, _SubQuerySupport __b);
+  template _SubQuerySupport& operator&=(_SubQuerySupport& __a, _SubQuerySupport __b);
+  template _SubQuerySupport& operator|=(_SubQuerySupport& __a, _SubQuerySupport __b);
+  template _SubQuerySupport& operator^=(_SubQuerySupport& __a, _SubQuerySupport __b);
 
-  template inline _SubQuerySupport operator~(_SubQuerySupport __a);
+  template _SubQuerySupport operator~(_SubQuerySupport __a);
 
   /**
    * SubQuery support flags
@@ -442,15 +442,15 @@ namespace meta {
       _U_UNION_ALL  = 1 << 2L //!< _U_UNION_ALL
   };
 
-  template inline _UnionSupport operator&(_UnionSupport __a, _UnionSupport __b);
-  template inline _UnionSupport operator|(_UnionSupport __a, _UnionSupport __b);
-  template inline _UnionSupport operator^(_UnionSupport __a, _UnionSupport __b);
+  template _UnionSupport operator&(_UnionSupport __a, _UnionSupport __b);
+  template _UnionSupport operator|(_UnionSupport __a, _UnionSupport __b);
+  template _UnionSupport operator^(_UnionSupport __a, _UnionSupport __b);
 
-  template inline _UnionSupport& operator&=(_UnionSupport& __a, _UnionSupport __b);
-  template inline _UnionSupport& operator|=(_UnionSupport& __a, _UnionSupport __b);
-  template inline _UnionSupport& operator^=(_UnionSupport& __a, _UnionSupport __b);
+  template _UnionSupport& operator&=(_UnionSupport& __a, _UnionSupport __b);
+  template _UnionSupport& operator|=(_UnionSupport& __a, _UnionSupport __b);
+  template _UnionSupport& operator^=(_UnionSupport& __a, _UnionSupport __b);
 
-  template inline _UnionSupport operator~(_UnionSupport __a);
+  template _UnionSupport operator~(_UnionSupport __a);
 
   /**
    * Union support flags
diff --git a/contrib/native/client/src/protobuf/CMakeLists.txt b/contrib/native/client/src/protobuf/CMakeLists.txt
index 95001c595e..b113234c7e 100644
--- a/contrib/native/client/src/protobuf/CMakeLists.txt
+++ b/contrib/native/client/src/protobuf/CMakeLists.txt
@@ -103,10 +103,6 @@ add_custom_target(cpProtobufs
 #message("ProtoHeaders =  ${ProtoHeaders}" )
 #message("ProtoIncludes =  ${ProtoIncludes}" )
 
-if(MSVC)
-    set(CMAKE_CXX_FLAGS "/EHsc")
-endif()
-
 add_library(protomsgs STATIC ${ProtoSources} ${ProtoHeaders} ${ProtoIncludes} )
 #set linker properties. The first time around, the protobufs generated files may not exist
 # and CMAKE will not be able to determine the linker type.
diff --git a/contrib/native/client/src/protobuf/UserBitShared.pb.cc b/contrib/native/client/src/protobuf/UserBitShared.pb.cc
index afab5afb95..c99030d95b 100644
--- a/contrib/native/client/src/protobuf/UserBitShared.pb.cc
+++ b/contrib/native/client/src/protobuf/UserBitShared.pb.cc
@@ -341,7 +341,7 @@ void protobuf_AssignDesc_UserBitShared_2eproto() {
       ::google::protobuf::MessageFactory::generated_factory(),
       sizeof(QueryInfo));
   QueryProfile_descriptor_ = file->message_type(13);
-  static const int QueryProfile_offsets_[21] = {
+  static const int QueryProfile_offsets_[22] = {
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(QueryProfile, id_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(QueryProfile, type_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(QueryProfile, start_),
@@ -363,6 +363,7 @@ void protobuf_AssignDesc_UserBitShared_2eproto() {
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(QueryProfile, queuewaitend_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(QueryProfile, total_cost_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(QueryProfile, queue_name_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(QueryProfile, queryid_),
   };
   QueryProfile_reflection_ =
     new ::google::protobuf::internal::GeneratedMessageReflection(
@@ -701,7 +702,7 @@ void protobuf_AddDesc_UserBitShared_2eproto() {
     "exec.shared.QueryResult.QueryState\022\017\n\004us"
     "er\030\004 \001(\t:\001-\022\'\n\007foreman\030\005 \001(\0132\026.exec.Dril"
     "lbitEndpoint\022\024\n\014options_json\030\006 \001(\t\022\022\n\nto"
-    "tal_cost\030\007 \001(\001\022\025\n\nqueue_name\030\010 \001(\t:\001-\"\242\004"
+    "tal_cost\030\007 \001(\001\022\025\n\nqueue_name\030\010 \001(\t:\001-\"\263\004"
     "\n\014QueryProfile\022 \n\002id\030\001 \001(\0132\024.exec.shared"
     ".QueryId\022$\n\004type\030\002 \001(\0162\026.exec.shared.Que"
     "ryType\022\r\n\005start\030\003 \001(\003\022\013\n\003end\030\004 \001(\003\022\r\n\005qu"
@@ -715,67 +716,67 @@ void protobuf_AddDesc_UserBitShared_2eproto() {
     " \001(\t\022\020\n\010error_id\030\017 \001(\t\022\022\n\nerror_node\030\020 \001"
     "(\t\022\024\n\014options_json\030\021 \001(\t\022\017\n\007planEnd\030\022 \001("
     "\003\022\024\n\014queueWaitEnd\030\023 \001(\003\022\022\n\ntotal_cost\030\024 "
-    "\001(\001\022\025\n\nqueue_name\030\025 \001(\t:\001-\"t\n\024MajorFragm"
-    "entProfile\022\031\n\021major_fragment_id\030\001 \001(\005\022A\n"
-    "\026minor_fragment_profile\030\002 \003(\0132!.exec.sha"
-    "red.MinorFragmentProfile\"\350\002\n\024MinorFragme"
-    "ntProfile\022)\n\005state\030\001 \001(\0162\032.exec.shared.F"
-    "ragmentState\022(\n\005error\030\002 \001(\0132\031.exec.share"
-    "d.DrillPBError\022\031\n\021minor_fragment_id\030\003 \001("
-    "\005\0226\n\020operator_profile\030\004 \003(\0132\034.exec.share"
-    "d.OperatorProfile\022\022\n\nstart_time\030\005 \001(\003\022\020\n"
-    "\010end_time\030\006 \001(\003\022\023\n\013memory_used\030\007 \001(\003\022\027\n\017"
-    "max_memory_used\030\010 \001(\003\022(\n\010endpoint\030\t \001(\0132"
-    "\026.exec.DrillbitEndpoint\022\023\n\013last_update\030\n"
-    " \001(\003\022\025\n\rlast_progress\030\013 \001(\003\"\377\001\n\017Operator"
-    "Profile\0221\n\rinput_profile\030\001 \003(\0132\032.exec.sh"
-    "ared.StreamProfile\022\023\n\013operator_id\030\003 \001(\005\022"
-    "\025\n\roperator_type\030\004 \001(\005\022\023\n\013setup_nanos\030\005 "
-    "\001(\003\022\025\n\rprocess_nanos\030\006 \001(\003\022#\n\033peak_local"
-    "_memory_allocated\030\007 \001(\003\022(\n\006metric\030\010 \003(\0132"
-    "\030.exec.shared.MetricValue\022\022\n\nwait_nanos\030"
-    "\t \001(\003\"B\n\rStreamProfile\022\017\n\007records\030\001 \001(\003\022"
-    "\017\n\007batches\030\002 \001(\003\022\017\n\007schemas\030\003 \001(\003\"J\n\013Met"
-    "ricValue\022\021\n\tmetric_id\030\001 \001(\005\022\022\n\nlong_valu"
-    "e\030\002 \001(\003\022\024\n\014double_value\030\003 \001(\001\")\n\010Registr"
-    "y\022\035\n\003jar\030\001 \003(\0132\020.exec.shared.Jar\"/\n\003Jar\022"
-    "\014\n\004name\030\001 \001(\t\022\032\n\022function_signature\030\002 \003("
-    "\t\"W\n\013SaslMessage\022\021\n\tmechanism\030\001 \001(\t\022\014\n\004d"
-    "ata\030\002 \001(\014\022\'\n\006status\030\003 \001(\0162\027.exec.shared."
-    "SaslStatus*5\n\nRpcChannel\022\017\n\013BIT_CONTROL\020"
-    "\000\022\014\n\010BIT_DATA\020\001\022\010\n\004USER\020\002*V\n\tQueryType\022\007"
-    "\n\003SQL\020\001\022\013\n\007LOGICAL\020\002\022\014\n\010PHYSICAL\020\003\022\r\n\tEX"
-    "ECUTION\020\004\022\026\n\022PREPARED_STATEMENT\020\005*\207\001\n\rFr"
-    "agmentState\022\013\n\007SENDING\020\000\022\027\n\023AWAITING_ALL"
-    "OCATION\020\001\022\013\n\007RUNNING\020\002\022\014\n\010FINISHED\020\003\022\r\n\t"
-    "CANCELLED\020\004\022\n\n\006FAILED\020\005\022\032\n\026CANCELLATION_"
-    "REQUESTED\020\006*\360\006\n\020CoreOperatorType\022\021\n\rSING"
-    "LE_SENDER\020\000\022\024\n\020BROADCAST_SENDER\020\001\022\n\n\006FIL"
-    "TER\020\002\022\022\n\016HASH_AGGREGATE\020\003\022\r\n\tHASH_JOIN\020\004"
-    "\022\016\n\nMERGE_JOIN\020\005\022\031\n\025HASH_PARTITION_SENDE"
-    "R\020\006\022\t\n\005LIMIT\020\007\022\024\n\020MERGING_RECEIVER\020\010\022\034\n\030"
-    "ORDERED_PARTITION_SENDER\020\t\022\013\n\007PROJECT\020\n\022"
-    "\026\n\022UNORDERED_RECEIVER\020\013\022\020\n\014RANGE_SENDER\020"
-    "\014\022\n\n\006SCREEN\020\r\022\034\n\030SELECTION_VECTOR_REMOVE"
-    "R\020\016\022\027\n\023STREAMING_AGGREGATE\020\017\022\016\n\nTOP_N_SO"
-    "RT\020\020\022\021\n\rEXTERNAL_SORT\020\021\022\t\n\005TRACE\020\022\022\t\n\005UN"
-    "ION\020\023\022\014\n\010OLD_SORT\020\024\022\032\n\026PARQUET_ROW_GROUP"
-    "_SCAN\020\025\022\021\n\rHIVE_SUB_SCAN\020\026\022\025\n\021SYSTEM_TAB"
-    "LE_SCAN\020\027\022\021\n\rMOCK_SUB_SCAN\020\030\022\022\n\016PARQUET_"
-    "WRITER\020\031\022\023\n\017DIRECT_SUB_SCAN\020\032\022\017\n\013TEXT_WR"
-    "ITER\020\033\022\021\n\rTEXT_SUB_SCAN\020\034\022\021\n\rJSON_SUB_SC"
-    "AN\020\035\022\030\n\024INFO_SCHEMA_SUB_SCAN\020\036\022\023\n\017COMPLE"
-    "X_TO_JSON\020\037\022\025\n\021PRODUCER_CONSUMER\020 \022\022\n\016HB"
-    "ASE_SUB_SCAN\020!\022\n\n\006WINDOW\020\"\022\024\n\020NESTED_LOO"
-    "P_JOIN\020#\022\021\n\rAVRO_SUB_SCAN\020$\022\021\n\rPCAP_SUB_"
-    "SCAN\020%\022\022\n\016KAFKA_SUB_SCAN\020&\022\021\n\rKUDU_SUB_S"
-    "CAN\020\'\022\013\n\007FLATTEN\020(\022\020\n\014LATERAL_JOIN\020)\022\n\n\006"
-    "UNNEST\020*\022,\n(HIVE_DRILL_NATIVE_PARQUET_RO"
-    "W_GROUP_SCAN\020+*g\n\nSaslStatus\022\020\n\014SASL_UNK"
-    "NOWN\020\000\022\016\n\nSASL_START\020\001\022\024\n\020SASL_IN_PROGRE"
-    "SS\020\002\022\020\n\014SASL_SUCCESS\020\003\022\017\n\013SASL_FAILED\020\004B"
-    ".\n\033org.apache.drill.exec.protoB\rUserBitS"
-    "haredH\001", 5167);
+    "\001(\001\022\025\n\nqueue_name\030\025 \001(\t:\001-\022\017\n\007queryId\030\026 "
+    "\001(\t\"t\n\024MajorFragmentProfile\022\031\n\021major_fra"
+    "gment_id\030\001 \001(\005\022A\n\026minor_fragment_profile"
+    "\030\002 \003(\0132!.exec.shared.MinorFragmentProfil"
+    "e\"\350\002\n\024MinorFragmentProfile\022)\n\005state\030\001 \001("
+    "\0162\032.exec.shared.FragmentState\022(\n\005error\030\002"
+    " \001(\0132\031.exec.shared.DrillPBError\022\031\n\021minor"
+    "_fragment_id\030\003 \001(\005\0226\n\020operator_profile\030\004"
+    " \003(\0132\034.exec.shared.OperatorProfile\022\022\n\nst"
+    "art_time\030\005 \001(\003\022\020\n\010end_time\030\006 \001(\003\022\023\n\013memo"
+    "ry_used\030\007 \001(\003\022\027\n\017max_memory_used\030\010 \001(\003\022("
+    "\n\010endpoint\030\t \001(\0132\026.exec.DrillbitEndpoint"
+    "\022\023\n\013last_update\030\n \001(\003\022\025\n\rlast_progress\030\013"
+    " \001(\003\"\377\001\n\017OperatorProfile\0221\n\rinput_profil"
+    "e\030\001 \003(\0132\032.exec.shared.StreamProfile\022\023\n\013o"
+    "perator_id\030\003 \001(\005\022\025\n\roperator_type\030\004 \001(\005\022"
+    "\023\n\013setup_nanos\030\005 \001(\003\022\025\n\rprocess_nanos\030\006 "
+    "\001(\003\022#\n\033peak_local_memory_allocated\030\007 \001(\003"
+    "\022(\n\006metric\030\010 \003(\0132\030.exec.shared.MetricVal"
+    "ue\022\022\n\nwait_nanos\030\t \001(\003\"B\n\rStreamProfile\022"
+    "\017\n\007records\030\001 \001(\003\022\017\n\007batches\030\002 \001(\003\022\017\n\007sch"
+    "emas\030\003 \001(\003\"J\n\013MetricValue\022\021\n\tmetric_id\030\001"
+    " \001(\005\022\022\n\nlong_value\030\002 \001(\003\022\024\n\014double_value"
+    "\030\003 \001(\001\")\n\010Registry\022\035\n\003jar\030\001 \003(\0132\020.exec.s"
+    "hared.Jar\"/\n\003Jar\022\014\n\004name\030\001 \001(\t\022\032\n\022functi"
+    "on_signature\030\002 \003(\t\"W\n\013SaslMessage\022\021\n\tmec"
+    "hanism\030\001 \001(\t\022\014\n\004data\030\002 \001(\014\022\'\n\006status\030\003 \001"
+    "(\0162\027.exec.shared.SaslStatus*5\n\nRpcChanne"
+    "l\022\017\n\013BIT_CONTROL\020\000\022\014\n\010BIT_DATA\020\001\022\010\n\004USER"
+    "\020\002*V\n\tQueryType\022\007\n\003SQL\020\001\022\013\n\007LOGICAL\020\002\022\014\n"
+    "\010PHYSICAL\020\003\022\r\n\tEXECUTION\020\004\022\026\n\022PREPARED_S"
+    "TATEMENT\020\005*\207\001\n\rFragmentState\022\013\n\007SENDING\020"
+    "\000\022\027\n\023AWAITING_ALLOCATION\020\001\022\013\n\007RUNNING\020\002\022"
+    "\014\n\010FINISHED\020\003\022\r\n\tCANCELLED\020\004\022\n\n\006FAILED\020\005"
+    "\022\032\n\026CANCELLATION_REQUESTED\020\006*\377\006\n\020CoreOpe"
+    "ratorType\022\021\n\rSINGLE_SENDER\020\000\022\024\n\020BROADCAS"
+    "T_SENDER\020\001\022\n\n\006FILTER\020\002\022\022\n\016HASH_AGGREGATE"
+    "\020\003\022\r\n\tHASH_JOIN\020\004\022\016\n\nMERGE_JOIN\020\005\022\031\n\025HAS"
+    "H_PARTITION_SENDER\020\006\022\t\n\005LIMIT\020\007\022\024\n\020MERGI"
+    "NG_RECEIVER\020\010\022\034\n\030ORDERED_PARTITION_SENDE"
+    "R\020\t\022\013\n\007PROJECT\020\n\022\026\n\022UNORDERED_RECEIVER\020\013"
+    "\022\020\n\014RANGE_SENDER\020\014\022\n\n\006SCREEN\020\r\022\034\n\030SELECT"
+    "ION_VECTOR_REMOVER\020\016\022\027\n\023STREAMING_AGGREG"
+    "ATE\020\017\022\016\n\nTOP_N_SORT\020\020\022\021\n\rEXTERNAL_SORT\020\021"
+    "\022\t\n\005TRACE\020\022\022\t\n\005UNION\020\023\022\014\n\010OLD_SORT\020\024\022\032\n\026"
+    "PARQUET_ROW_GROUP_SCAN\020\025\022\021\n\rHIVE_SUB_SCA"
+    "N\020\026\022\025\n\021SYSTEM_TABLE_SCAN\020\027\022\021\n\rMOCK_SUB_S"
+    "CAN\020\030\022\022\n\016PARQUET_WRITER\020\031\022\023\n\017DIRECT_SUB_"
+    "SCAN\020\032\022\017\n\013TEXT_WRITER\020\033\022\021\n\rTEXT_SUB_SCAN"
+    "\020\034\022\021\n\rJSON_SUB_SCAN\020\035\022\030\n\024INFO_SCHEMA_SUB"
+    "_SCAN\020\036\022\023\n\017COMPLEX_TO_JSON\020\037\022\025\n\021PRODUCER"
+    "_CONSUMER\020 \022\022\n\016HBASE_SUB_SCAN\020!\022\n\n\006WINDO"
+    "W\020\"\022\024\n\020NESTED_LOOP_JOIN\020#\022\021\n\rAVRO_SUB_SC"
+    "AN\020$\022\021\n\rPCAP_SUB_SCAN\020%\022\022\n\016KAFKA_SUB_SCA"
+    "N\020&\022\021\n\rKUDU_SUB_SCAN\020\'\022\013\n\007FLATTEN\020(\022\020\n\014L"
+    "ATERAL_JOIN\020)\022\n\n\006UNNEST\020*\022,\n(HIVE_DRILL_"
+    "NATIVE_PARQUET_ROW_GROUP_SCAN\020+\022\r\n\tJDBC_"
+    "SCAN\020,*g\n\nSaslStatus\022\020\n\014SASL_UNKNOWN\020\000\022\016"
+    "\n\nSASL_START\020\001\022\024\n\020SASL_IN_PROGRESS\020\002\022\020\n\014"
+    "SASL_SUCCESS\020\003\022\017\n\013SASL_FAILED\020\004B.\n\033org.a"
+    "pache.drill.exec.protoB\rUserBitSharedH\001", 5199);
   ::google::protobuf::MessageFactory::InternalRegisterGeneratedFile(
     "UserBitShared.proto", &protobuf_RegisterTypes);
   UserCredentials::default_instance_ = new UserCredentials();
@@ -940,6 +941,7 @@ bool CoreOperatorType_IsValid(int value) {
     case 41:
     case 42:
     case 43:
+    case 44:
       return true;
     default:
       return false;
@@ -5562,6 +5564,7 @@ const int QueryProfile::kPlanEndFieldNumber;
 const int QueryProfile::kQueueWaitEndFieldNumber;
 const int QueryProfile::kTotalCostFieldNumber;
 const int QueryProfile::kQueueNameFieldNumber;
+const int QueryProfile::kQueryIdFieldNumber;
 #endif  // !_MSC_VER
 
 QueryProfile::QueryProfile()
@@ -5602,6 +5605,7 @@ void QueryProfile::SharedCtor() {
   queuewaitend_ = GOOGLE_LONGLONG(0);
   total_cost_ = 0;
   queue_name_ = const_cast< ::std::string*>(_default_queue_name_);
+  queryid_ = const_cast< ::std::string*>(&::google::protobuf::internal::kEmptyString);
   ::memset(_has_bits_, 0, sizeof(_has_bits_));
 }
 
@@ -5637,6 +5641,9 @@ void QueryProfile::SharedDtor() {
   if (queue_name_ != _default_queue_name_) {
     delete queue_name_;
   }
+  if (queryid_ != &::google::protobuf::internal::kEmptyString) {
+    delete queryid_;
+  }
   if (this != default_instance_) {
     delete id_;
     delete foreman_;
@@ -5730,6 +5737,11 @@ void QueryProfile::Clear() {
         queue_name_->assign(*_default_queue_name_);
       }
     }
+    if (has_queryid()) {
+      if (queryid_ != &::google::protobuf::internal::kEmptyString) {
+        queryid_->clear();
+      }
+    }
   }
   fragment_profile_.Clear();
   ::memset(_has_bits_, 0, sizeof(_has_bits_));
@@ -6087,6 +6099,23 @@ bool QueryProfile::MergePartialFromCodedStream(
         } else {
           goto handle_uninterpreted;
         }
+        if (input->ExpectTag(178)) goto parse_queryId;
+        break;
+      }
+
+      // optional string queryId = 22;
+      case 22: {
+        if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
+            ::google::protobuf::internal::WireFormatLite::WIRETYPE_LENGTH_DELIMITED) {
+         parse_queryId:
+          DO_(::google::protobuf::internal::WireFormatLite::ReadString(
+                input, this->mutable_queryid()));
+          ::google::protobuf::internal::WireFormat::VerifyUTF8String(
+            this->queryid().data(), this->queryid().length(),
+            ::google::protobuf::internal::WireFormat::PARSE);
+        } else {
+          goto handle_uninterpreted;
+        }
         if (input->ExpectAtEnd()) return true;
         break;
       }
@@ -6255,6 +6284,15 @@ void QueryProfile::SerializeWithCachedSizes(
       21, this->queue_name(), output);
   }
 
+  // optional string queryId = 22;
+  if (has_queryid()) {
+    ::google::protobuf::internal::WireFormat::VerifyUTF8String(
+      this->queryid().data(), this->queryid().length(),
+      ::google::protobuf::internal::WireFormat::SERIALIZE);
+    ::google::protobuf::internal::WireFormatLite::WriteString(
+      22, this->queryid(), output);
+  }
+
   if (!unknown_fields().empty()) {
     ::google::protobuf::internal::WireFormat::SerializeUnknownFields(
         unknown_fields(), output);
@@ -6421,6 +6459,16 @@ ::google::protobuf::uint8* QueryProfile::SerializeWithCachedSizesToArray(
         21, this->queue_name(), target);
   }
 
+  // optional string queryId = 22;
+  if (has_queryid()) {
+    ::google::protobuf::internal::WireFormat::VerifyUTF8String(
+      this->queryid().data(), this->queryid().length(),
+      ::google::protobuf::internal::WireFormat::SERIALIZE);
+    target =
+      ::google::protobuf::internal::WireFormatLite::WriteStringToArray(
+        22, this->queryid(), target);
+  }
+
   if (!unknown_fields().empty()) {
     target = ::google::protobuf::internal::WireFormat::SerializeUnknownFieldsToArray(
         unknown_fields(), target);
@@ -6572,6 +6620,13 @@ int QueryProfile::ByteSize() const {
           this->queue_name());
     }
 
+    // optional string queryId = 22;
+    if (has_queryid()) {
+      total_size += 2 +
+        ::google::protobuf::internal::WireFormatLite::StringSize(
+          this->queryid());
+    }
+
   }
   // repeated .exec.shared.MajorFragmentProfile fragment_profile = 11;
   total_size += 1 * this->fragment_profile_size();
@@ -6672,6 +6727,9 @@ void QueryProfile::MergeFrom(const QueryProfile& from) {
     if (from.has_queue_name()) {
       set_queue_name(from.queue_name());
     }
+    if (from.has_queryid()) {
+      set_queryid(from.queryid());
+    }
   }
   mutable_unknown_fields()->MergeFrom(from.unknown_fields());
 }
@@ -6716,6 +6774,7 @@ void QueryProfile::Swap(QueryProfile* other) {
     std::swap(queuewaitend_, other->queuewaitend_);
     std::swap(total_cost_, other->total_cost_);
     std::swap(queue_name_, other->queue_name_);
+    std::swap(queryid_, other->queryid_);
     std::swap(_has_bits_[0], other->_has_bits_[0]);
     _unknown_fields_.Swap(&other->_unknown_fields_);
     std::swap(_cached_size_, other->_cached_size_);
diff --git a/contrib/native/client/src/protobuf/UserBitShared.pb.h b/contrib/native/client/src/protobuf/UserBitShared.pb.h
index 66c4cf7986..a78258a9ad 100644
--- a/contrib/native/client/src/protobuf/UserBitShared.pb.h
+++ b/contrib/native/client/src/protobuf/UserBitShared.pb.h
@@ -247,11 +247,12 @@ enum CoreOperatorType {
   FLATTEN = 40,
   LATERAL_JOIN = 41,
   UNNEST = 42,
-  HIVE_DRILL_NATIVE_PARQUET_ROW_GROUP_SCAN = 43
+  HIVE_DRILL_NATIVE_PARQUET_ROW_GROUP_SCAN = 43,
+  JDBC_SCAN = 44
 };
 bool CoreOperatorType_IsValid(int value);
 const CoreOperatorType CoreOperatorType_MIN = SINGLE_SENDER;
-const CoreOperatorType CoreOperatorType_MAX = HIVE_DRILL_NATIVE_PARQUET_ROW_GROUP_SCAN;
+const CoreOperatorType CoreOperatorType_MAX = JDBC_SCAN;
 const int CoreOperatorType_ARRAYSIZE = CoreOperatorType_MAX + 1;
 
 const ::google::protobuf::EnumDescriptor* CoreOperatorType_descriptor();
@@ -2172,6 +2173,18 @@ class QueryProfile : public ::google::protobuf::Message {
   inline ::std::string* release_queue_name();
   inline void set_allocated_queue_name(::std::string* queue_name);
 
+  // optional string queryId = 22;
+  inline bool has_queryid() const;
+  inline void clear_queryid();
+  static const int kQueryIdFieldNumber = 22;
+  inline const ::std::string& queryid() const;
+  inline void set_queryid(const ::std::string& value);
+  inline void set_queryid(const char* value);
+  inline void set_queryid(const char* value, size_t size);
+  inline ::std::string* mutable_queryid();
+  inline ::std::string* release_queryid();
+  inline void set_allocated_queryid(::std::string* queryid);
+
   // @@protoc_insertion_point(class_scope:exec.shared.QueryProfile)
  private:
   inline void set_has_id();
@@ -2214,6 +2227,8 @@ class QueryProfile : public ::google::protobuf::Message {
   inline void clear_has_total_cost();
   inline void set_has_queue_name();
   inline void clear_has_queue_name();
+  inline void set_has_queryid();
+  inline void clear_has_queryid();
 
   ::google::protobuf::UnknownFieldSet _unknown_fields_;
 
@@ -2240,9 +2255,10 @@ class QueryProfile : public ::google::protobuf::Message {
   double total_cost_;
   ::std::string* queue_name_;
   static ::std::string* _default_queue_name_;
+  ::std::string* queryid_;
 
   mutable int _cached_size_;
-  ::google::protobuf::uint32 _has_bits_[(21 + 31) / 32];
+  ::google::protobuf::uint32 _has_bits_[(22 + 31) / 32];
 
   friend void  protobuf_AddDesc_UserBitShared_2eproto();
   friend void protobuf_AssignDesc_UserBitShared_2eproto();
@@ -6091,6 +6107,76 @@ inline void QueryProfile::set_allocated_queue_name(::std::string* queue_name) {
   }
 }
 
+// optional string queryId = 22;
+inline bool QueryProfile::has_queryid() const {
+  return (_has_bits_[0] & 0x00200000u) != 0;
+}
+inline void QueryProfile::set_has_queryid() {
+  _has_bits_[0] |= 0x00200000u;
+}
+inline void QueryProfile::clear_has_queryid() {
+  _has_bits_[0] &= ~0x00200000u;
+}
+inline void QueryProfile::clear_queryid() {
+  if (queryid_ != &::google::protobuf::internal::kEmptyString) {
+    queryid_->clear();
+  }
+  clear_has_queryid();
+}
+inline const ::std::string& QueryProfile::queryid() const {
+  return *queryid_;
+}
+inline void QueryProfile::set_queryid(const ::std::string& value) {
+  set_has_queryid();
+  if (queryid_ == &::google::protobuf::internal::kEmptyString) {
+    queryid_ = new ::std::string;
+  }
+  queryid_->assign(value);
+}
+inline void QueryProfile::set_queryid(const char* value) {
+  set_has_queryid();
+  if (queryid_ == &::google::protobuf::internal::kEmptyString) {
+    queryid_ = new ::std::string;
+  }
+  queryid_->assign(value);
+}
+inline void QueryProfile::set_queryid(const char* value, size_t size) {
+  set_has_queryid();
+  if (queryid_ == &::google::protobuf::internal::kEmptyString) {
+    queryid_ = new ::std::string;
+  }
+  queryid_->assign(reinterpret_cast<const char*>(value), size);
+}
+inline ::std::string* QueryProfile::mutable_queryid() {
+  set_has_queryid();
+  if (queryid_ == &::google::protobuf::internal::kEmptyString) {
+    queryid_ = new ::std::string;
+  }
+  return queryid_;
+}
+inline ::std::string* QueryProfile::release_queryid() {
+  clear_has_queryid();
+  if (queryid_ == &::google::protobuf::internal::kEmptyString) {
+    return NULL;
+  } else {
+    ::std::string* temp = queryid_;
+    queryid_ = const_cast< ::std::string*>(&::google::protobuf::internal::kEmptyString);
+    return temp;
+  }
+}
+inline void QueryProfile::set_allocated_queryid(::std::string* queryid) {
+  if (queryid_ != &::google::protobuf::internal::kEmptyString) {
+    delete queryid_;
+  }
+  if (queryid) {
+    set_has_queryid();
+    queryid_ = queryid;
+  } else {
+    clear_has_queryid();
+    queryid_ = const_cast< ::std::string*>(&::google::protobuf::internal::kEmptyString);
+  }
+}
+
 // -------------------------------------------------------------------
 
 // MajorFragmentProfile
diff --git a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/CompareFunctionsProcessor.java b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/CompareFunctionsProcessor.java
index 0672b53c09..59df4e2cc7 100644
--- a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/CompareFunctionsProcessor.java
+++ b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/CompareFunctionsProcessor.java
@@ -22,6 +22,8 @@
 
 import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 
 import org.apache.drill.common.expression.CastExpression;
 import org.apache.drill.common.expression.ConvertExpression;
@@ -51,6 +53,10 @@
 import com.google.common.collect.ImmutableSet;
 
 public class CompareFunctionsProcessor extends AbstractExprVisitor<Boolean, LogicalExpression, RuntimeException> {
+
+  // to check that function name starts with convert_from disregarding the case and has encoding after
+  private static final Pattern convertFromPattern = Pattern.compile(String.format("^%s(.+)", ConvertExpression.CONVERT_FROM), Pattern.CASE_INSENSITIVE);
+
   private byte[] value;
   private boolean success;
   private boolean isEqualityFn;
@@ -511,6 +517,17 @@ public Boolean visitSchemaPath(SchemaPath path, LogicalExpression valueArg) thro
     return false;
   }
 
+  @Override
+  public Boolean visitFunctionCall(FunctionCall call, LogicalExpression valueArg) {
+    Matcher matcher = convertFromPattern.matcher(call.getName());
+    if (matcher.find()) {
+      // convert function call to ConvertExpression
+      ConvertExpression convert = new ConvertExpression(ConvertExpression.CONVERT_FROM, matcher.group(1), call.args.get(0), call.getPosition());
+      return visitConvertExpression(convert, valueArg);
+    }
+    return false;
+  }
+
   protected static ByteBuf newByteBuf(int size, boolean bigEndian) {
     return Unpooled.wrappedBuffer(new byte[size])
         .order(bigEndian ? ByteOrder.BIG_ENDIAN : ByteOrder.LITTLE_ENDIAN)
diff --git a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseFilterBuilder.java b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseFilterBuilder.java
index 8d2e8ffd7c..6e1efe512c 100644
--- a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseFilterBuilder.java
+++ b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseFilterBuilder.java
@@ -61,6 +61,7 @@ public HBaseScanSpec parseTree() {
        * remove it since its effect is also achieved through startRow and stopRow.
        */
       if (parsedSpec.filter instanceof RowFilter &&
+          ((RowFilter)parsedSpec.filter).getOperator() != CompareOp.NOT_EQUAL &&
           ((RowFilter)parsedSpec.filter).getComparator() instanceof BinaryComparator) {
         parsedSpec.filter = null;
       }
diff --git a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseGroupScan.java b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseGroupScan.java
index 9eeba245fc..97c9a95ac0 100644
--- a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseGroupScan.java
+++ b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseGroupScan.java
@@ -22,7 +22,9 @@
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Iterator;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -63,20 +65,13 @@
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Stopwatch;
-import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
 
 @JsonTypeName("hbase-scan")
 public class HBaseGroupScan extends AbstractGroupScan implements DrillHBaseConstants {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HBaseGroupScan.class);
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HBaseGroupScan.class);
 
-  private static final Comparator<List<HBaseSubScanSpec>> LIST_SIZE_COMPARATOR = new Comparator<List<HBaseSubScanSpec>>() {
-    @Override
-    public int compare(List<HBaseSubScanSpec> list1, List<HBaseSubScanSpec> list2) {
-      return list1.size() - list2.size();
-    }
-  };
+  private static final Comparator<List<HBaseSubScanSpec>> LIST_SIZE_COMPARATOR = (list1, list2) -> list1.size() - list2.size();
 
   private static final Comparator<List<HBaseSubScanSpec>> LIST_SIZE_COMPARATOR_REV = Collections.reverseOrder(LIST_SIZE_COMPARATOR);
 
@@ -182,12 +177,12 @@ private void init() {
   public List<EndpointAffinity> getOperatorAffinity() {
     watch.reset();
     watch.start();
-    Map<String, DrillbitEndpoint> endpointMap = new HashMap<String, DrillbitEndpoint>();
+    Map<String, DrillbitEndpoint> endpointMap = new HashMap<>();
     for (DrillbitEndpoint ep : storagePlugin.getContext().getBits()) {
       endpointMap.put(ep.getAddress(), ep);
     }
 
-    Map<DrillbitEndpoint, EndpointAffinity> affinityMap = new HashMap<DrillbitEndpoint, EndpointAffinity>();
+    Map<DrillbitEndpoint, EndpointAffinity> affinityMap = new HashMap<>();
     for (ServerName sn : regionsToScan.values()) {
       DrillbitEndpoint ep = endpointMap.get(sn.getHostname());
       if (ep != null) {
@@ -199,14 +194,10 @@ private void init() {
         }
       }
     }
-    logger.debug("Took {} µs to get operator affinity", watch.elapsed(TimeUnit.NANOSECONDS)/1000);
-    return Lists.newArrayList(affinityMap.values());
+    logger.debug("Took {} µs to get operator affinity", watch.elapsed(TimeUnit.NANOSECONDS) / 1000);
+    return new ArrayList<>(affinityMap.values());
   }
 
-  /**
-   *
-   * @param incomingEndpoints
-   */
   @Override
   public void applyAssignments(List<DrillbitEndpoint> incomingEndpoints) {
     watch.reset();
@@ -230,23 +221,23 @@ public void applyAssignments(List<DrillbitEndpoint> incomingEndpoints) {
     /*
      * another map with endpoint (hostname => corresponding index list) in 'incomingEndpoints' list
      */
-    Map<String, Queue<Integer>> endpointHostIndexListMap = Maps.newHashMap();
+    Map<String, Queue<Integer>> endpointHostIndexListMap = new HashMap<>();
 
     /*
      * Initialize these two maps
      */
     for (int i = 0; i < numSlots; ++i) {
-      endpointFragmentMapping.put(i, new ArrayList<HBaseSubScanSpec>(maxPerEndpointSlot));
+      endpointFragmentMapping.put(i, new ArrayList<>(maxPerEndpointSlot));
       String hostname = incomingEndpoints.get(i).getAddress();
       Queue<Integer> hostIndexQueue = endpointHostIndexListMap.get(hostname);
       if (hostIndexQueue == null) {
-        hostIndexQueue = Lists.newLinkedList();
+        hostIndexQueue = new LinkedList<>();
         endpointHostIndexListMap.put(hostname, hostIndexQueue);
       }
       hostIndexQueue.add(i);
     }
 
-    Set<Entry<HRegionInfo, ServerName>> regionsToAssignSet = Sets.newHashSet(regionsToScan.entrySet());
+    Set<Entry<HRegionInfo, ServerName>> regionsToAssignSet = new HashSet<>(regionsToScan.entrySet());
 
     /*
      * First, we assign regions which are hosted on region servers running on drillbit endpoints
@@ -256,13 +247,13 @@ public void applyAssignments(List<DrillbitEndpoint> incomingEndpoints) {
       /*
        * Test if there is a drillbit endpoint which is also an HBase RegionServer that hosts the current HBase region
        */
-      Queue<Integer> endpointIndexlist = endpointHostIndexListMap.get(regionEntry.getValue().getHostname());
-      if (endpointIndexlist != null) {
-        Integer slotIndex = endpointIndexlist.poll();
+      Queue<Integer> endpointIndexList = endpointHostIndexListMap.get(regionEntry.getValue().getHostname());
+      if (endpointIndexList != null) {
+        Integer slotIndex = endpointIndexList.poll();
         List<HBaseSubScanSpec> endpointSlotScanList = endpointFragmentMapping.get(slotIndex);
         endpointSlotScanList.add(regionInfoToSubScanSpec(regionEntry.getKey()));
         // add to the tail of the slot list, to add more later in round robin fashion
-        endpointIndexlist.offer(slotIndex);
+        endpointIndexList.offer(slotIndex);
         // this region has been assigned
         regionsIterator.remove();
       }
@@ -271,8 +262,8 @@ public void applyAssignments(List<DrillbitEndpoint> incomingEndpoints) {
     /*
      * Build priority queues of slots, with ones which has tasks lesser than 'minPerEndpointSlot' and another which have more.
      */
-    PriorityQueue<List<HBaseSubScanSpec>> minHeap = new PriorityQueue<List<HBaseSubScanSpec>>(numSlots, LIST_SIZE_COMPARATOR);
-    PriorityQueue<List<HBaseSubScanSpec>> maxHeap = new PriorityQueue<List<HBaseSubScanSpec>>(numSlots, LIST_SIZE_COMPARATOR_REV);
+    PriorityQueue<List<HBaseSubScanSpec>> minHeap = new PriorityQueue<>(numSlots, LIST_SIZE_COMPARATOR);
+    PriorityQueue<List<HBaseSubScanSpec>> maxHeap = new PriorityQueue<>(numSlots, LIST_SIZE_COMPARATOR_REV);
     for(List<HBaseSubScanSpec> listOfScan : endpointFragmentMapping.values()) {
       if (listOfScan.size() < minPerEndpointSlot) {
         minHeap.offer(listOfScan);
@@ -310,12 +301,11 @@ public void applyAssignments(List<DrillbitEndpoint> incomingEndpoints) {
     }
 
     /* no slot should be empty at this point */
-    assert (minHeap.peek() == null || minHeap.peek().size() > 0) : String.format(
-        "Unable to assign tasks to some endpoints.\nEndpoints: {}.\nAssignment Map: {}.",
-        incomingEndpoints, endpointFragmentMapping.toString());
+    assert (minHeap.peek() == null || minHeap.peek().size() > 0) :
+      String.format("Unable to assign tasks to some endpoints.\nEndpoints: %s.\nAssignment Map: %s.", incomingEndpoints, endpointFragmentMapping.toString());
 
     logger.debug("Built assignment map in {} µs.\nEndpoints: {}.\nAssignment Map: {}",
-        watch.elapsed(TimeUnit.NANOSECONDS)/1000, incomingEndpoints, endpointFragmentMapping.toString());
+        watch.elapsed(TimeUnit.NANOSECONDS) / 1000, incomingEndpoints, endpointFragmentMapping.toString());
   }
 
   private HBaseSubScanSpec regionInfoToSubScanSpec(HRegionInfo ri) {
@@ -347,9 +337,15 @@ public int getMaxParallelizationWidth() {
 
   @Override
   public ScanStats getScanStats() {
-    long rowCount = (long) ((scanSizeInBytes / statsCalculator.getAvgRowSizeInBytes()) * (hbaseScanSpec.getFilter() != null ? 0.5 : 1));
-    // the following calculation is not precise since 'columns' could specify CFs while getColsPerRow() returns the number of qualifier.
-    float diskCost = scanSizeInBytes * ((columns == null || columns.isEmpty()) ? 1 : columns.size()/statsCalculator.getColsPerRow());
+    long rowCount = scanSizeInBytes / statsCalculator.getAvgRowSizeInBytes();
+    // the following calculation is not precise since 'columns' could specify CFs while getColsPerRow() returns the number of qualifier
+    float diskCost = scanSizeInBytes * ((columns == null || columns.isEmpty()) ? 1 : columns.size() / statsCalculator.getColsPerRow());
+    // if filter push down is used, reduce estimated row count and disk cost by half to ensure plan cost will be less then without it
+    if (hbaseScanSpec.getFilter() != null) {
+      rowCount = (long) (rowCount * 0.5);
+      // if during sampling we found out exact row count, no need to reduce number of rows
+      diskCost = statsCalculator.usedDefaultRowCount() ? diskCost * 0.5F : diskCost;
+    }
     return new ScanStats(GroupScanProperty.NO_EXACT_ROW_COUNT, rowCount, 1, diskCost);
   }
 
diff --git a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/TableStatsCalculator.java b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/TableStatsCalculator.java
index 379fb7cd47..b435fbdb0a 100644
--- a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/TableStatsCalculator.java
+++ b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/TableStatsCalculator.java
@@ -48,36 +48,36 @@
  * Computes size of each region for given table.
  */
 public class TableStatsCalculator {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TableStatsCalculator.class);
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TableStatsCalculator.class);
 
-  public static final long DEFAULT_ROW_COUNT = 1024L * 1024L;
+  public static final long DEFAULT_ROW_COUNT = 1024L * 1024L; // 1 million rows
 
   private static final String DRILL_EXEC_HBASE_SCAN_SAMPLE_ROWS_COUNT = "drill.exec.hbase.scan.samplerows.count";
 
   private static final int DEFAULT_SAMPLE_SIZE = 100;
 
-  /**
-   * Maps each region to its size in bytes.
-   */
+  // Maps each region to its size in bytes.
   private Map<byte[], Long> sizeMap = null;
 
   private int avgRowSizeInBytes = 1;
 
   private int colsPerRow = 1;
 
+  private long estimatedRowCount = DEFAULT_ROW_COUNT;
+
   /**
    * Computes size of each region for table.
    *
-   * @param conn
-   * @param hbaseScanSpec
-   * @param config
-   * @throws IOException
+   * @param connection connection to Hbase client
+   * @param hbaseScanSpec scan specification
+   * @param config drill configuration
+   * @param storageConfig Hbase storage configuration
    */
-  public TableStatsCalculator(Connection conn, HBaseScanSpec hbaseScanSpec, DrillConfig config, HBaseStoragePluginConfig storageConfig) throws IOException {
+  public TableStatsCalculator(Connection connection, HBaseScanSpec hbaseScanSpec, DrillConfig config, HBaseStoragePluginConfig storageConfig) throws IOException {
     TableName tableName = TableName.valueOf(hbaseScanSpec.getTableName());
-    try (Admin admin = conn.getAdmin();
-         Table table = conn.getTable(tableName);
-         RegionLocator locator = conn.getRegionLocator(tableName)) {
+    try (Admin admin = connection.getAdmin();
+         Table table = connection.getTable(tableName);
+         RegionLocator locator = connection.getRegionLocator(tableName)) {
       int rowsToSample = rowsToSample(config);
       if (rowsToSample > 0) {
         Scan scan = new Scan(hbaseScanSpec.getStartRow(), hbaseScanSpec.getStopRow());
@@ -100,22 +100,25 @@ public TableStatsCalculator(Connection conn, HBaseScanSpec hbaseScanSpec, DrillC
           }
         }
         if (rowCount > 0) {
-          avgRowSizeInBytes = (int) (rowSizeSum/rowCount);
-          colsPerRow = numColumnsSum/rowCount;
+          avgRowSizeInBytes = (int) (rowSizeSum / rowCount);
+          colsPerRow = numColumnsSum / rowCount;
+          // if during sampling we receive less rows than expected, then we can use this number instead of default
+          estimatedRowCount = rowCount == rowsToSample ? estimatedRowCount : rowCount;
         }
+
         scanner.close();
       }
 
       if (!enabled(storageConfig)) {
-        logger.info("Region size calculation disabled.");
+        logger.debug("Region size calculation is disabled.");
         return;
       }
 
-      logger.info("Calculating region sizes for table '{}'.", tableName.getNameAsString());
+      logger.debug("Calculating region sizes for table '{}'.", tableName.getNameAsString());
 
-      //get regions for table
+      // get regions for table
       List<HRegionLocation> tableRegionInfos = locator.getAllRegionLocations();
-      Set<byte[]> tableRegions = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
+      Set<byte[]> tableRegions = new TreeSet<>(Bytes.BYTES_COMPARATOR);
       for (HRegionLocation regionInfo : tableRegionInfos) {
         tableRegions.add(regionInfo.getRegionInfo().getRegionName());
       }
@@ -124,17 +127,17 @@ public TableStatsCalculator(Connection conn, HBaseScanSpec hbaseScanSpec, DrillC
       try {
         clusterStatus = admin.getClusterStatus();
       } catch (Exception e) {
-        logger.debug(e.getMessage());
+        logger.debug(e.getMessage(), e);
       } finally {
         if (clusterStatus == null) {
           return;
         }
       }
 
-      sizeMap = new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
+      sizeMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
 
       Collection<ServerName> servers = clusterStatus.getServers();
-      //iterate all cluster regions, filter regions from our table and compute their size
+      // iterate all cluster regions, filter regions from our table and compute their size
       for (ServerName serverName : servers) {
         ServerLoad serverLoad = clusterStatus.getLoad(serverName);
 
@@ -143,14 +146,12 @@ public TableStatsCalculator(Connection conn, HBaseScanSpec hbaseScanSpec, DrillC
 
           if (tableRegions.contains(regionId)) {
             long regionSizeMB = regionLoad.getMemStoreSizeMB() + regionLoad.getStorefileSizeMB();
-            sizeMap.put(regionId, (regionSizeMB > 0 ? regionSizeMB : 1) * (1024*1024));
-            if (logger.isDebugEnabled()) {
-              logger.debug("Region " + regionLoad.getNameAsString() + " has size " + regionSizeMB + "MB");
-            }
+            sizeMap.put(regionId, (regionSizeMB > 0 ? regionSizeMB : 1) * estimatedRowCount);
+            logger.debug("Region {} has size {} MB.", regionLoad.getNameAsString(), regionSizeMB);
           }
         }
       }
-      logger.debug("Region sizes calculated");
+      logger.debug("Region sizes calculated.");
     }
 
   }
@@ -160,8 +161,8 @@ private boolean enabled(HBaseStoragePluginConfig config) {
   }
 
   private int rowsToSample(DrillConfig config) {
-    return config.hasPath(DRILL_EXEC_HBASE_SCAN_SAMPLE_ROWS_COUNT)
-        ? config.getInt(DRILL_EXEC_HBASE_SCAN_SAMPLE_ROWS_COUNT) : DEFAULT_SAMPLE_SIZE;
+    return config.hasPath(DRILL_EXEC_HBASE_SCAN_SAMPLE_ROWS_COUNT) ?
+      config.getInt(DRILL_EXEC_HBASE_SCAN_SAMPLE_ROWS_COUNT) : DEFAULT_SAMPLE_SIZE;
   }
 
   /**
@@ -169,11 +170,11 @@ private int rowsToSample(DrillConfig config) {
    */
   public long getRegionSizeInBytes(byte[] regionId) {
     if (sizeMap == null) {
-      return (long) avgRowSizeInBytes * DEFAULT_ROW_COUNT; // 1 million rows
+      return (long) avgRowSizeInBytes * estimatedRowCount;
     } else {
       Long size = sizeMap.get(regionId);
       if (size == null) {
-        logger.debug("Unknown region:" + Arrays.toString(regionId));
+        logger.debug("Unknown region: {}.", Arrays.toString(regionId));
         return 0;
       } else {
         return size;
@@ -189,4 +190,8 @@ public int getColsPerRow() {
     return colsPerRow;
   }
 
+  public boolean usedDefaultRowCount() {
+    return estimatedRowCount == DEFAULT_ROW_COUNT;
+  }
+
 }
diff --git a/contrib/storage-hbase/src/main/resources/bootstrap-storage-plugins.json b/contrib/storage-hbase/src/main/resources/bootstrap-storage-plugins.json
index 3e0e8c0417..530a407c84 100644
--- a/contrib/storage-hbase/src/main/resources/bootstrap-storage-plugins.json
+++ b/contrib/storage-hbase/src/main/resources/bootstrap-storage-plugins.json
@@ -2,11 +2,11 @@
   "storage":{
     hbase : {
       type:"hbase",
-      enabled: false,
       config : {
         "hbase.zookeeper.quorum" : "localhost",
         "hbase.zookeeper.property.clientPort" : 2181
-      }
+      },
+      enabled: false
     }
   }
 }
diff --git a/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/BaseHBaseTest.java b/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/BaseHBaseTest.java
index 51312fc392..dd7ce674e4 100644
--- a/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/BaseHBaseTest.java
+++ b/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/BaseHBaseTest.java
@@ -84,7 +84,6 @@ protected void runHBasePhysicalVerifyCount(String planFile, String tableName, in
 
   protected List<QueryDataBatch> runHBaseSQLlWithResults(String sql) throws Exception {
     sql = canonizeHBaseSQL(sql);
-    System.out.println("Running query:\n" + sql);
     return testSqlWithResults(sql);
   }
 
diff --git a/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/TestHBaseFilterPushDown.java b/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/TestHBaseFilterPushDown.java
index 0e14cb183e..aee18eb704 100644
--- a/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/TestHBaseFilterPushDown.java
+++ b/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/TestHBaseFilterPushDown.java
@@ -44,6 +44,24 @@ public void testFilterPushDownRowKeyEqual() throws Exception {
     PlanTestBase.testPlanMatchingPatterns(sqlHBase, expectedPlan, excludedPlan);
   }
 
+  @Test
+  public void testFilterPushDownRowKeyNotEqual() throws Exception {
+    setColumnWidths(new int[] {8, 38, 38});
+    final String sql = "SELECT\n"
+        + "  *\n"
+        + "FROM\n"
+        + "  hbase.`[TABLE_NAME]` tableName\n"
+        + "WHERE\n"
+        + "  row_key <> 'b4'";
+
+    runHBaseSQLVerifyCount(sql, 7);
+
+    final String[] expectedPlan = {".*startRow=, stopRow=, filter=RowFilter \\(NOT_EQUAL, b4\\).*"};
+    final String[] excludedPlan ={};
+    final String sqlHBase = canonizeHBaseSQL(sql);
+    PlanTestBase.testPlanMatchingPatterns(sqlHBase, expectedPlan, excludedPlan);
+  }
+
   @Test
   public void testFilterPushDownRowKeyEqualWithItem() throws Exception {
     setColumnWidths(new int[] {20, 30});
@@ -780,5 +798,21 @@ public void testDummyColumnsAreAvoided() throws Exception {
     runHBaseSQLVerifyCount(sql, 2);
   }
 
+  @Test
+  public void testConvertFromPushDownWithView() throws Exception {
+    test("create view dfs.tmp.pd_view as\n" +
+       "select convert_from(byte_substr(row_key, 1, 8), 'date_epoch_be') as d\n" +
+       "from hbase.`TestTableCompositeDate`");
+
+    String query = "select d from dfs.tmp.pd_view where d > date '2015-06-13' and d < DATE '2015-06-18'";
+    String[] expectedPlan = {
+        "startRow=\\\\x00\\\\x00\\\\x01M\\\\xEF\\]\\\\xA0\\\\x00, " +
+        "stopRow=\\\\x00\\\\x00\\\\x01N\\\\x03\\\\xF7\\\\x10\\\\x00, " +
+        "filter=null"};
+    String[] excludedPlan ={"Filter\\("};
+    PlanTestBase.testPlanMatchingPatterns(query, expectedPlan, excludedPlan);
+
+    runHBaseSQLVerifyCount(query, 12);
+  }
 }
 
diff --git a/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/TestHBaseTableProvider.java b/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/TestHBaseTableProvider.java
index ca6856483e..1a9f4ee924 100644
--- a/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/TestHBaseTableProvider.java
+++ b/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/TestHBaseTableProvider.java
@@ -21,8 +21,6 @@
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-import java.util.Map.Entry;
-
 import com.google.common.collect.Lists;
 import org.apache.drill.categories.HbaseStorageTest;
 import org.apache.drill.common.config.LogicalPlanPersistence;
@@ -66,12 +64,7 @@ public void testTableProvider() throws StoreException {
     assertTrue(hbaseStore.contains(""));
     assertFalse(hbaseStore.contains("unknown_key"));
 
-    int rowCount = 0;
-    for (Entry<String, String> entry : Lists.newArrayList(hbaseStore.getAll())) {
-      rowCount++;
-      System.out.println(entry.getKey() + "=" + entry.getValue());
-    }
-    assertEquals(7, rowCount);
+    assertEquals(7, Lists.newArrayList(hbaseStore.getAll()).size());
 
     PersistentStore<String> hbaseTestStore = provider.getOrCreateStore(PersistentStoreConfig.newJacksonBuilder(lp.getMapper(), String.class).name("hbase.test").build());
     hbaseTestStore.put("", "v0");
@@ -84,12 +77,7 @@ public void testTableProvider() throws StoreException {
     assertEquals("v0", hbaseStore.get(""));
     assertEquals("testValue", hbaseStore.get(".test"));
 
-    rowCount = 0;
-    for (Entry<String, String> entry : Lists.newArrayList(hbaseTestStore.getAll())) {
-      rowCount++;
-      System.out.println(entry.getKey() + "=" + entry.getValue());
-    }
-    assertEquals(6, rowCount);
+    assertEquals(6, Lists.newArrayList(hbaseTestStore.getAll()).size());
   }
 
   @AfterClass
diff --git a/contrib/storage-hive/core/pom.xml b/contrib/storage-hive/core/pom.xml
index 0f7ff1f088..e0dbb5ac0f 100644
--- a/contrib/storage-hive/core/pom.xml
+++ b/contrib/storage-hive/core/pom.xml
@@ -142,6 +142,11 @@
         </exclusion>
       </exclusions>
     </dependency>
+    <dependency>
+      <groupId>org.apache.drill.contrib</groupId>
+      <artifactId>drill-format-mapr</artifactId>
+      <version>${project.version}</version>
+    </dependency>
   </dependencies>
 
   <build>
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/expr/fn/HiveFuncHolder.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/expr/fn/HiveFuncHolder.java
index 8e7b645b0a..80f299ecdf 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/expr/fn/HiveFuncHolder.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/expr/fn/HiveFuncHolder.java
@@ -130,7 +130,7 @@ public int getParamCount() {
    * @return workspace variables
    */
   @Override
-  public JVar[] renderStart(ClassGenerator<?> g, HoldingContainer[] inputVariables){
+  public JVar[] renderStart(ClassGenerator<?> g, HoldingContainer[] inputVariables, FieldReference fieldReference){
     JVar[] workspaceJVars = new JVar[5];
 
     workspaceJVars[0] = g.declareClassField("returnOI", g.getModel()._ref(ObjectInspector.class));
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/logical/ConvertHiveMapRDBJsonScanToDrillMapRDBJsonScan.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/logical/ConvertHiveMapRDBJsonScanToDrillMapRDBJsonScan.java
new file mode 100644
index 0000000000..3bc33b331d
--- /dev/null
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/logical/ConvertHiveMapRDBJsonScanToDrillMapRDBJsonScan.java
@@ -0,0 +1,186 @@
+/*
+ * 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.planner.sql.logical;
+
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.planner.logical.DrillScanRel;
+import org.apache.drill.exec.planner.logical.RelOptHelper;
+import org.apache.drill.exec.store.StoragePluginOptimizerRule;
+import org.apache.drill.exec.store.hive.HiveMetadataProvider;
+import org.apache.drill.exec.store.hive.HiveReadEntry;
+import org.apache.drill.exec.store.hive.HiveScan;
+import org.apache.drill.exec.store.mapr.db.MapRDBFormatPlugin;
+import org.apache.drill.exec.store.mapr.db.MapRDBFormatPluginConfig;
+import org.apache.drill.exec.store.mapr.db.json.JsonScanSpec;
+import org.apache.drill.exec.store.mapr.db.json.JsonTableGroupScan;
+import org.ojai.DocumentConstants;
+
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static org.apache.drill.exec.store.hive.HiveUtilities.nativeReadersRuleMatches;
+
+/**
+ * Convert Hive scan to use Drill's native MapR-DB reader instead of Hive's MapR-DB JSON Handler.
+ */
+public class ConvertHiveMapRDBJsonScanToDrillMapRDBJsonScan extends StoragePluginOptimizerRule {
+  private static final org.slf4j.Logger logger =
+      org.slf4j.LoggerFactory.getLogger(ConvertHiveMapRDBJsonScanToDrillMapRDBJsonScan.class);
+
+  public static final ConvertHiveMapRDBJsonScanToDrillMapRDBJsonScan INSTANCE =
+      new ConvertHiveMapRDBJsonScanToDrillMapRDBJsonScan();
+
+  /**
+   * The constants from org.apache.hadoop.hive.maprdb.json.conf.MapRDBConstants
+   */
+  private static final String MAPRDB_PFX = "maprdb.";
+  private static final String MAPRDB_TABLE_NAME = MAPRDB_PFX + "table.name";
+  private static final String ID_KEY = DocumentConstants.ID_KEY;
+  private static final String MAPRDB_COLUMN_ID = MAPRDB_PFX + "column.id";
+
+  private ConvertHiveMapRDBJsonScanToDrillMapRDBJsonScan() {
+    super(RelOptHelper.any(DrillScanRel.class), "ConvertHiveScanToHiveDrillNativeScan:MapR-DB");
+  }
+
+  /**
+   * {@see org.apache.drill.exec.store.hive.HiveUtilities#nativeReadersRuleMatches}
+   */
+  @Override
+  public boolean matches(RelOptRuleCall call) {
+    try {
+      return nativeReadersRuleMatches(call,
+          Class.forName("org.apache.hadoop.hive.maprdb.json.input.HiveMapRDBJsonInputFormat"));
+    } catch (ClassNotFoundException e) {
+      throw UserException.resourceError(e)
+          .message("Current Drill build is not designed for working with Hive MapR-DB tables. " +
+              "Please disable \"%s\" option", ExecConstants.HIVE_OPTIMIZE_MAPRDB_JSON_SCAN_WITH_NATIVE_READER)
+          .build(logger);
+    }
+  }
+
+  @Override
+  public void onMatch(RelOptRuleCall call) {
+    try {
+      DrillScanRel hiveScanRel = call.rel(0);
+
+      HiveScan hiveScan = (HiveScan) hiveScanRel.getGroupScan();
+      HiveReadEntry hiveReadEntry = hiveScan.getHiveReadEntry();
+      HiveMetadataProvider hiveMetadataProvider = new HiveMetadataProvider(hiveScan.getUserName(), hiveReadEntry,
+          hiveScan.getHiveConf());
+      if (hiveMetadataProvider.getInputSplits(hiveReadEntry).isEmpty()) {
+        // table is empty, use original scan
+        return;
+      }
+
+      if (hiveScan.getHiveReadEntry().getTable().isSetPartitionKeys()) {
+        logger.warn("Hive MapR-DB JSON Handler doesn't support table partitioning. Consider recreating table without " +
+            "partitions");
+      }
+
+      DrillScanRel nativeScanRel = createNativeScanRel(hiveScanRel);
+      call.transformTo(nativeScanRel);
+
+      /*
+        Drill native scan should take precedence over Hive since it's more efficient and faster.
+        Hive does not always give correct costing (i.e. for external tables Hive does not have number of rows
+        and we calculate them approximately). On the contrary, Drill calculates number of rows exactly
+        and thus Hive Scan can be chosen instead of Drill native scan because costings allegedly lower for Hive.
+        To ensure Drill MapR-DB Json scan will be chosen, reduce Hive scan importance to 0.
+       */
+      call.getPlanner().setImportance(hiveScanRel, 0.0);
+    } catch (final Exception e) {
+      logger.warn("Failed to convert HiveScan to JsonScanSpec", e);
+    }
+  }
+
+  /**
+   * Helper method which creates a DrillScanRel with native Drill HiveScan.
+   */
+  private DrillScanRel createNativeScanRel(final DrillScanRel hiveScanRel) throws Exception {
+    RelDataTypeFactory typeFactory = hiveScanRel.getCluster().getTypeFactory();
+    HiveScan hiveScan = (HiveScan) hiveScanRel.getGroupScan();
+    Map<String, String> parameters = hiveScan.getHiveReadEntry().getHiveTableWrapper().getParameters();
+
+    JsonScanSpec scanSpec = new JsonScanSpec(parameters.get(MAPRDB_TABLE_NAME), null);
+    MapRDBFormatPlugin mapRDBFormatPlugin = new MapRDBFormatPlugin(
+        "hive-maprdb",
+        hiveScan.getStoragePlugin().getContext(),
+        hiveScan.getHiveConf(),
+        hiveScan.getStoragePlugin().getConfig(),
+        new MapRDBFormatPluginConfig()
+    );
+    List<SchemaPath> hiveScanCols = hiveScanRel.getColumns().stream()
+        .map(colNameSchemaPath -> replaceOverriddenSchemaPath(parameters, colNameSchemaPath))
+        .collect(Collectors.toList());
+    JsonTableGroupScan nativeMapRDBScan =
+        new JsonTableGroupScan(
+            hiveScan.getUserName(),
+            hiveScan.getStoragePlugin(),
+            mapRDBFormatPlugin,
+            scanSpec,
+            hiveScanCols
+        );
+
+    List<String> nativeScanColNames = hiveScanRel.getRowType().getFieldList().stream()
+        .map(field -> replaceOverriddenColumnId(parameters, field.getName()))
+        .collect(Collectors.toList());
+    List<RelDataType> nativeScanColTypes = hiveScanRel.getRowType().getFieldList().stream()
+        .map(RelDataTypeField::getType)
+        .collect(Collectors.toList());
+    RelDataType nativeScanRowType = typeFactory.createStructType(nativeScanColTypes, nativeScanColNames);
+
+    return new DrillScanRel(
+        hiveScanRel.getCluster(),
+        hiveScanRel.getTraitSet(),
+        hiveScanRel.getTable(),
+        nativeMapRDBScan,
+        nativeScanRowType,
+        hiveScanCols);
+  }
+
+  /**
+   * Hive maps column id "_id" with custom user column id name. Replace it for {@link DrillScanRel}
+   *
+   * @param parameters Hive table properties
+   * @param colName Hive column name
+   * @return original column name, null if colName is absent
+   */
+  private String replaceOverriddenColumnId(Map<String, String> parameters, String colName) {
+    return colName != null && colName.equals(parameters.get(MAPRDB_COLUMN_ID)) ? ID_KEY : colName;
+  }
+
+  /**
+   * The same as above, but for {@link SchemaPath} object
+   *
+   * @param parameters Hive table properties
+   * @param colNameSchemaPath SchemaPath with Hive column name
+   * @return SchemaPath with original column name
+   */
+  private SchemaPath replaceOverriddenSchemaPath(Map<String, String> parameters, SchemaPath colNameSchemaPath) {
+    String hiveColumnName = colNameSchemaPath.getRootSegmentPath();
+    return hiveColumnName != null && hiveColumnName.equals(parameters.get(MAPRDB_COLUMN_ID))
+        ? SchemaPath.getSimplePath(ID_KEY) : colNameSchemaPath;
+  }
+}
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/logical/ConvertHiveParquetScanToDrillParquetScan.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/logical/ConvertHiveParquetScanToDrillParquetScan.java
index 3484ab32a6..ea711572dc 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/logical/ConvertHiveParquetScanToDrillParquetScan.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/logical/ConvertHiveParquetScanToDrillParquetScan.java
@@ -17,9 +17,6 @@
  */
 package org.apache.drill.exec.planner.sql.logical;
 
-import com.google.common.base.Strings;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
@@ -40,22 +37,16 @@
 import org.apache.drill.exec.store.hive.HiveMetadataProvider;
 import org.apache.drill.exec.store.hive.HiveReadEntry;
 import org.apache.drill.exec.store.hive.HiveScan;
-import org.apache.drill.exec.store.hive.HiveTableWithColumnCache;
-import org.apache.drill.exec.store.hive.HiveTableWrapper.HivePartitionWrapper;
-import org.apache.drill.exec.store.hive.HiveUtilities;
-import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
-import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
-import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.mapred.JobConf;
 
+import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Properties;
+
+import static org.apache.drill.exec.store.hive.HiveUtilities.nativeReadersRuleMatches;
 
 /**
  * Convert Hive scan to use Drill's native parquet reader instead of Hive's native reader. It also adds a
@@ -78,94 +69,11 @@ private ConvertHiveParquetScanToDrillParquetScan() {
   }
 
   /**
-   * Rule is matched when all of the following match:
-   * 1) GroupScan in given DrillScalRel is an {@link HiveScan}
-   * 2) {@link HiveScan} is not already rewritten using Drill's native readers
-   * 3) InputFormat in Hive table metadata and all partitions metadata contains the same value
-   *    {@link MapredParquetInputFormat}
-   * 4) No error occurred while checking for the above conditions. An error is logged as warning.
-   *
-   * @param call rule call
-   * @return True if the rule can be applied. False otherwise
+   * {@see org.apache.drill.exec.store.hive.HiveUtilities#nativeReadersRuleMatches}
    */
   @Override
   public boolean matches(RelOptRuleCall call) {
-    final DrillScanRel scanRel = call.rel(0);
-
-    if (!(scanRel.getGroupScan() instanceof HiveScan) || ((HiveScan) scanRel.getGroupScan()).isNativeReader()) {
-      return false;
-    }
-
-    final HiveScan hiveScan = (HiveScan) scanRel.getGroupScan();
-    final HiveConf hiveConf = hiveScan.getHiveConf();
-    final HiveTableWithColumnCache hiveTable = hiveScan.getHiveReadEntry().getTable();
-
-    if (containsUnsupportedDataTypes(hiveTable)) {
-      return false;
-    }
-
-    final Class<? extends InputFormat<?,?>> tableInputFormat =
-        getInputFormatFromSD(HiveUtilities.getTableMetadata(hiveTable), hiveScan.getHiveReadEntry(), hiveTable.getSd(),
-            hiveConf);
-    if (tableInputFormat == null || !tableInputFormat.equals(MapredParquetInputFormat.class)) {
-      return false;
-    }
-
-    final List<HivePartitionWrapper> partitions = hiveScan.getHiveReadEntry().getHivePartitionWrappers();
-    if (partitions == null) {
-      return true;
-    }
-
-    final List<FieldSchema> tableSchema = hiveTable.getSd().getCols();
-    // Make sure all partitions have the same input format as the table input format
-    for (HivePartitionWrapper partition : partitions) {
-      final StorageDescriptor partitionSD = partition.getPartition().getSd();
-      Class<? extends InputFormat<?, ?>> inputFormat = getInputFormatFromSD(
-          HiveUtilities.getPartitionMetadata(partition.getPartition(), hiveTable), hiveScan.getHiveReadEntry(), partitionSD,
-          hiveConf);
-      if (inputFormat == null || !inputFormat.equals(tableInputFormat)) {
-        return false;
-      }
-
-      // Make sure the schema of the table and schema of the partition matches. If not return false. Schema changes
-      // between table and partition can happen when table schema is altered using ALTER statements after some
-      // partitions are already created. Currently native reader conversion doesn't handle schema changes between
-      // partition and table. Hive has extensive list of convert methods to convert from one type to rest of the
-      // possible types. Drill doesn't have the similar set of methods yet.
-      if (!partitionSD.getCols().equals(tableSchema)) {
-        logger.debug("Partitions schema is different from table schema. Currently native reader conversion can't " +
-            "handle schema difference between partitions and table");
-        return false;
-      }
-    }
-
-    return true;
-  }
-
-  /**
-   * Get the input format from given {@link StorageDescriptor}
-   * @param properties table properties
-   * @param hiveReadEntry hive read entry
-   * @param sd storage descriptor
-   * @return {@link InputFormat} class or null if a failure has occurred. Failure is logged as warning.
-   */
-  private Class<? extends InputFormat<?, ?>> getInputFormatFromSD(final Properties properties,
-      final HiveReadEntry hiveReadEntry, final StorageDescriptor sd, final HiveConf hiveConf) {
-    final Table hiveTable = hiveReadEntry.getTable();
-    try {
-      final String inputFormatName = sd.getInputFormat();
-      if (!Strings.isNullOrEmpty(inputFormatName)) {
-        return (Class<? extends InputFormat<?, ?>>) Class.forName(inputFormatName);
-      }
-
-      final JobConf job = new JobConf(hiveConf);
-      HiveUtilities.addConfToJob(job, properties);
-      return HiveUtilities.getInputFormatClass(job, sd, hiveTable);
-    } catch (final Exception e) {
-      logger.warn("Failed to get InputFormat class from Hive table '{}.{}'. StorageDescriptor [{}]",
-          hiveTable.getDbName(), hiveTable.getTableName(), sd.toString(), e);
-      return null;
-    }
+    return nativeReadersRuleMatches(call, MapredParquetInputFormat.class);
   }
 
   @Override
@@ -180,7 +88,7 @@ public void onMatch(RelOptRuleCall call) {
       final Table hiveTable = hiveScan.getHiveReadEntry().getTable();
       final HiveReadEntry hiveReadEntry = hiveScan.getHiveReadEntry();
 
-      final HiveMetadataProvider hiveMetadataProvider = new HiveMetadataProvider(hiveScan.getUserName(), hiveReadEntry, hiveScan.getStoragePlugin().getHiveConf());
+      final HiveMetadataProvider hiveMetadataProvider = new HiveMetadataProvider(hiveScan.getUserName(), hiveReadEntry, hiveScan.getHiveConf());
       final List<HiveMetadataProvider.LogicalInputSplit> logicalInputSplits = hiveMetadataProvider.getInputSplits(hiveReadEntry);
 
       if (logicalInputSplits.isEmpty()) {
@@ -203,7 +111,7 @@ public void onMatch(RelOptRuleCall call) {
         Hive does not always give correct costing (i.e. for external tables Hive does not have number of rows
         and we calculate them approximately). On the contrary, Drill calculates number of rows exactly
         and thus Hive Scan can be chosen instead of Drill native scan because costings allegedly lower for Hive.
-        To ensure Drill native scan we'll be chosen, reduce Hive scan importance to 0.
+        To ensure Drill native scan will be chosen, reduce Hive scan importance to 0.
        */
       call.getPlanner().setImportance(hiveScanRel, 0.0);
     } catch (final Exception e) {
@@ -215,7 +123,7 @@ Hive does not always give correct costing (i.e. for external tables Hive does no
    * Create mapping of Hive partition column to directory column mapping.
    */
   private Map<String, String> getPartitionColMapping(final Table hiveTable, final String partitionColumnLabel) {
-    final Map<String, String> partitionColMapping = Maps.newHashMap();
+    final Map<String, String> partitionColMapping = new HashMap<>();
     int i = 0;
     for (FieldSchema col : hiveTable.getPartitionKeys()) {
       partitionColMapping.put(col.getName(), partitionColumnLabel+i);
@@ -235,8 +143,8 @@ private DrillScanRel createNativeScanRel(final Map<String, String> partitionColM
     final RelDataTypeFactory typeFactory = hiveScanRel.getCluster().getTypeFactory();
     final RelDataType varCharType = typeFactory.createSqlType(SqlTypeName.VARCHAR);
 
-    final List<String> nativeScanColNames = Lists.newArrayList();
-    final List<RelDataType> nativeScanColTypes = Lists.newArrayList();
+    final List<String> nativeScanColNames = new ArrayList<>();
+    final List<RelDataType> nativeScanColTypes = new ArrayList<>();
     for (RelDataTypeField field : hiveScanRel.getRowType().getFieldList()) {
       final String dirColName = partitionColMapping.get(field.getName());
       if (dirColName != null) { // partition column
@@ -253,8 +161,8 @@ private DrillScanRel createNativeScanRel(final Map<String, String> partitionColM
     // Create the list of projected columns set in HiveScan. The order of this list may not be same as the order of
     // columns in HiveScan row type. Note: If the HiveScan.getColumn() contains a '*', we just need to add it as it is,
     // unlike above where we expanded the '*'. HiveScan and related (subscan) can handle '*'.
-    final List<SchemaPath> nativeScanCols = Lists.newArrayList();
-    for(SchemaPath colName : hiveScanRel.getColumns()) {
+    final List<SchemaPath> nativeScanCols = new ArrayList<>();
+    for (SchemaPath colName : hiveScanRel.getColumns()) {
       final String partitionCol = partitionColMapping.get(colName.getRootSegmentPath());
       if (partitionCol != null) {
         nativeScanCols.add(SchemaPath.getSimplePath(partitionCol));
@@ -269,7 +177,8 @@ private DrillScanRel createNativeScanRel(final Map<String, String> partitionColM
             hiveScan.getUserName(),
             nativeScanCols,
             hiveScan.getStoragePlugin(),
-            logicalInputSplits);
+            logicalInputSplits,
+            hiveScan.getConfProperties());
 
     return new DrillScanRel(
         hiveScanRel.getCluster(),
@@ -286,7 +195,7 @@ private DrillScanRel createNativeScanRel(final Map<String, String> partitionColM
   private DrillProjectRel createProjectRel(final DrillScanRel hiveScanRel,
       final Map<String, String> partitionColMapping, final DrillScanRel nativeScanRel) {
 
-    final List<RexNode> rexNodes = Lists.newArrayList();
+    final List<RexNode> rexNodes = new ArrayList<>();
     final RexBuilder rb = hiveScanRel.getCluster().getRexBuilder();
     final RelDataType hiveScanRowType = hiveScanRel.getRowType();
 
@@ -341,18 +250,4 @@ private RexNode createPartitionColumnCast(final DrillScanRel hiveScanRel, final
 
     return rb.makeCast(outputType, inputRef);
   }
-
-  private boolean containsUnsupportedDataTypes(final Table hiveTable) {
-    for (FieldSchema hiveField : hiveTable.getSd().getCols()) {
-      final Category category = TypeInfoUtils.getTypeInfoFromTypeString(hiveField.getType()).getCategory();
-      if (category == Category.MAP ||
-          category == Category.STRUCT ||
-          category == Category.UNION ||
-          category == Category.LIST) {
-        logger.debug("Hive table contains unsupported data type: {}", category);
-        return true;
-      }
-    }
-    return false;
-  }
 }
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetRowGroupScan.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetRowGroupScan.java
index e22701511a..d334ec8b4f 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetRowGroupScan.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetRowGroupScan.java
@@ -38,6 +38,7 @@
 
 import java.io.IOException;
 import java.util.List;
+import java.util.Map;
 
 @JsonTypeName("hive-drill-native-parquet-row-group-scan")
 public class HiveDrillNativeParquetRowGroupScan extends AbstractParquetRowGroupScan {
@@ -45,6 +46,7 @@
   private final HiveStoragePlugin hiveStoragePlugin;
   private final HiveStoragePluginConfig hiveStoragePluginConfig;
   private final HivePartitionHolder hivePartitionHolder;
+  private final Map<String, String> confProperties;
 
   @JsonCreator
   public HiveDrillNativeParquetRowGroupScan(@JacksonInject StoragePluginRegistry registry,
@@ -53,12 +55,14 @@ public HiveDrillNativeParquetRowGroupScan(@JacksonInject StoragePluginRegistry r
                                             @JsonProperty("rowGroupReadEntries") List<RowGroupReadEntry> rowGroupReadEntries,
                                             @JsonProperty("columns") List<SchemaPath> columns,
                                             @JsonProperty("hivePartitionHolder") HivePartitionHolder hivePartitionHolder,
+                                            @JsonProperty("confProperties") Map<String, String> confProperties,
                                             @JsonProperty("filter") LogicalExpression filter) throws ExecutionSetupException {
     this(userName,
         (HiveStoragePlugin) registry.getPlugin(hiveStoragePluginConfig),
         rowGroupReadEntries,
         columns,
         hivePartitionHolder,
+        confProperties,
         filter);
   }
 
@@ -67,11 +71,13 @@ public HiveDrillNativeParquetRowGroupScan(String userName,
                                             List<RowGroupReadEntry> rowGroupReadEntries,
                                             List<SchemaPath> columns,
                                             HivePartitionHolder hivePartitionHolder,
+                                            Map<String, String> confProperties,
                                             LogicalExpression filter) {
     super(userName, rowGroupReadEntries, columns, filter);
     this.hiveStoragePlugin = Preconditions.checkNotNull(hiveStoragePlugin, "Could not find format config for the given configuration");
     this.hiveStoragePluginConfig = hiveStoragePlugin.getConfig();
     this.hivePartitionHolder = hivePartitionHolder;
+    this.confProperties = confProperties;
   }
 
   @JsonProperty
@@ -84,6 +90,11 @@ public HivePartitionHolder getHivePartitionHolder() {
     return hivePartitionHolder;
   }
 
+  @JsonProperty
+  public Map<String, String> getConfProperties() {
+    return confProperties;
+  }
+
   @JsonIgnore
   public HiveStoragePlugin getHiveStoragePlugin() {
     return hiveStoragePlugin;
@@ -92,7 +103,7 @@ public HiveStoragePlugin getHiveStoragePlugin() {
   @Override
   public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
     Preconditions.checkArgument(children.isEmpty());
-    return new HiveDrillNativeParquetRowGroupScan(getUserName(), hiveStoragePlugin, rowGroupReadEntries, columns, hivePartitionHolder, filter);
+    return new HiveDrillNativeParquetRowGroupScan(getUserName(), hiveStoragePlugin, rowGroupReadEntries, columns, hivePartitionHolder, confProperties, filter);
   }
 
   @Override
@@ -102,7 +113,7 @@ public int getOperatorType() {
 
   @Override
   public AbstractParquetRowGroupScan copy(List<SchemaPath> columns) {
-    return new HiveDrillNativeParquetRowGroupScan(getUserName(), hiveStoragePlugin, rowGroupReadEntries, columns, hivePartitionHolder, filter);
+    return new HiveDrillNativeParquetRowGroupScan(getUserName(), hiveStoragePlugin, rowGroupReadEntries, columns, hivePartitionHolder, confProperties, filter);
   }
 
   @Override
@@ -114,7 +125,7 @@ public boolean areCorruptDatesAutoCorrected() {
   public Configuration getFsConf(RowGroupReadEntry rowGroupReadEntry) throws IOException {
     Path path = new Path(rowGroupReadEntry.getPath()).getParent();
     return new ProjectionPusher().pushProjectionsAndFilters(
-        new JobConf(hiveStoragePlugin.getHiveConf()),
+        new JobConf(HiveUtilities.generateHiveConf(hiveStoragePlugin.getHiveConf(), confProperties)),
         path.getParent());
   }
 
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetScan.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetScan.java
index 03a80d3386..a973fa12b8 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetScan.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetScan.java
@@ -62,7 +62,8 @@
 public class HiveDrillNativeParquetScan extends AbstractParquetGroupScan {
 
   private final HiveStoragePlugin hiveStoragePlugin;
-  private HivePartitionHolder hivePartitionHolder;
+  private final HivePartitionHolder hivePartitionHolder;
+  private final Map<String, String> confProperties;
 
   @JsonCreator
   public HiveDrillNativeParquetScan(@JacksonInject StoragePluginRegistry engineRegistry,
@@ -71,10 +72,12 @@ public HiveDrillNativeParquetScan(@JacksonInject StoragePluginRegistry engineReg
                                     @JsonProperty("columns") List<SchemaPath> columns,
                                     @JsonProperty("entries") List<ReadEntryWithPath> entries,
                                     @JsonProperty("hivePartitionHolder") HivePartitionHolder hivePartitionHolder,
+                                    @JsonProperty("confProperties") Map<String, String> confProperties,
                                     @JsonProperty("filter") LogicalExpression filter) throws IOException, ExecutionSetupException {
     super(ImpersonationUtil.resolveUserName(userName), columns, entries, filter);
     this.hiveStoragePlugin = (HiveStoragePlugin) engineRegistry.getPlugin(hiveStoragePluginConfig);
     this.hivePartitionHolder = hivePartitionHolder;
+    this.confProperties = confProperties;
 
     init();
   }
@@ -82,19 +85,22 @@ public HiveDrillNativeParquetScan(@JacksonInject StoragePluginRegistry engineReg
   public HiveDrillNativeParquetScan(String userName,
                                     List<SchemaPath> columns,
                                     HiveStoragePlugin hiveStoragePlugin,
-                                    List<LogicalInputSplit> logicalInputSplits) throws IOException {
-    this(userName, columns, hiveStoragePlugin, logicalInputSplits, ValueExpressions.BooleanExpression.TRUE);
+                                    List<LogicalInputSplit> logicalInputSplits,
+                                    Map<String, String> confProperties) throws IOException {
+    this(userName, columns, hiveStoragePlugin, logicalInputSplits, confProperties, ValueExpressions.BooleanExpression.TRUE);
   }
 
   public HiveDrillNativeParquetScan(String userName,
                                     List<SchemaPath> columns,
                                     HiveStoragePlugin hiveStoragePlugin,
                                     List<LogicalInputSplit> logicalInputSplits,
+                                    Map<String, String> confProperties,
                                     LogicalExpression filter) throws IOException {
     super(userName, columns, new ArrayList<>(), filter);
 
     this.hiveStoragePlugin = hiveStoragePlugin;
     this.hivePartitionHolder = new HivePartitionHolder();
+    this.confProperties = confProperties;
 
     for (LogicalInputSplit logicalInputSplit : logicalInputSplits) {
       Iterator<InputSplit> iterator = logicalInputSplit.getInputSplits().iterator();
@@ -122,6 +128,7 @@ private HiveDrillNativeParquetScan(HiveDrillNativeParquetScan that) {
     super(that);
     this.hiveStoragePlugin = that.hiveStoragePlugin;
     this.hivePartitionHolder = that.hivePartitionHolder;
+    this.confProperties = that.confProperties;
   }
 
   @JsonProperty
@@ -134,6 +141,11 @@ public HivePartitionHolder getHivePartitionHolder() {
     return hivePartitionHolder;
   }
 
+  @JsonProperty
+  public Map<String, String> getConfProperties() {
+    return confProperties;
+  }
+
   @Override
   public SubScan getSpecificScan(int minorFragmentId) {
     List<RowGroupReadEntry> readEntries = getReadEntries(minorFragmentId);
@@ -142,7 +154,7 @@ public SubScan getSpecificScan(int minorFragmentId) {
       List<String> values = hivePartitionHolder.get(readEntry.getPath());
       subPartitionHolder.add(readEntry.getPath(), values);
     }
-    return new HiveDrillNativeParquetRowGroupScan(getUserName(), hiveStoragePlugin, readEntries, columns, subPartitionHolder, filter);
+    return new HiveDrillNativeParquetRowGroupScan(getUserName(), hiveStoragePlugin, readEntries, columns, subPartitionHolder, confProperties, filter);
   }
 
   @Override
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveMetadataProvider.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveMetadataProvider.java
index c8775643a1..de45dc6eb4 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveMetadataProvider.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveMetadataProvider.java
@@ -19,8 +19,6 @@
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Stopwatch;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
 import com.google.common.collect.Multimap;
 import com.google.common.collect.Ordering;
 import com.google.common.collect.TreeMultimap;
@@ -50,12 +48,14 @@
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
 /**
  * Class which provides methods to get metadata of given Hive table selection. It tries to use the stats stored in
@@ -79,50 +79,52 @@
   public HiveMetadataProvider(final String userName, final HiveReadEntry hiveReadEntry, final HiveConf hiveConf) {
     this.hiveReadEntry = hiveReadEntry;
     this.ugi = ImpersonationUtil.createProxyUgi(userName);
-    isPartitionedTable = hiveReadEntry.getTable().getPartitionKeysSize() > 0;
-    partitionInputSplitMap = Maps.newHashMap();
+    this.isPartitionedTable = hiveReadEntry.getTable().getPartitionKeysSize() > 0;
+    this.partitionInputSplitMap = new HashMap<>();
     this.hiveConf = hiveConf;
   }
 
   /**
-   * Return stats for table/partitions in given {@link HiveReadEntry}. If valid stats are available in MetaStore,
-   * return it. Otherwise estimate using the size of the input data.
+   * Return stats for table/partitions in given {@link HiveReadEntry}.
+   * If valid stats are available in MetaStore, return it.
+   * Otherwise estimate using the size of the input data.
    *
    * @param hiveReadEntry Subset of the {@link HiveReadEntry} used when creating this cache object.
-   * @return
-   * @throws IOException
+   * @return hive statistics holder
+   * @throws IOException if was unable to retrieve table statistics
    */
   public HiveStats getStats(final HiveReadEntry hiveReadEntry) throws IOException {
-    final Stopwatch timeGetStats = Stopwatch.createStarted();
+    Stopwatch timeGetStats = Stopwatch.createStarted();
 
-    final HiveTableWithColumnCache table = hiveReadEntry.getTable();
+    HiveTableWithColumnCache table = hiveReadEntry.getTable();
     try {
       if (!isPartitionedTable) {
-        final Properties properties = MetaStoreUtils.getTableMetadata(table);
-        final HiveStats stats = getStatsFromProps(properties);
+        Properties properties = MetaStoreUtils.getTableMetadata(table);
+        HiveStats stats = HiveStats.getStatsFromProps(properties);
         if (stats.valid()) {
           return stats;
         }
 
-        // estimate the stats from the InputSplits.
-        return getStatsEstimateFromInputSplits(getTableInputSplits());
+        return stats.getSizeInBytes() > 0 ? estimateStatsFromBytes(stats.getSizeInBytes()) :
+            estimateStatsFromInputSplits(getTableInputSplits());
+
       } else {
-        final HiveStats aggStats = new HiveStats(0, 0);
-        for(HivePartition partition : hiveReadEntry.getPartitions()) {
-          final Properties properties = HiveUtilities.getPartitionMetadata(partition, table);
-          HiveStats stats = getStatsFromProps(properties);
+        HiveStats aggStats = new HiveStats(0, 0);
+        for (HivePartition partition : hiveReadEntry.getPartitions()) {
+          Properties properties = HiveUtilities.getPartitionMetadata(partition, table);
+          HiveStats stats = HiveStats.getStatsFromProps(properties);
 
           if (!stats.valid()) {
-            // estimate the stats from InputSplits
-            stats = getStatsEstimateFromInputSplits(getPartitionInputSplits(partition));
+            stats = stats.getSizeInBytes() > 0 ? estimateStatsFromBytes(stats.getSizeInBytes()) :
+                estimateStatsFromInputSplits(getPartitionInputSplits(partition));
           }
           aggStats.add(stats);
         }
 
         return aggStats;
       }
-    } catch (final Exception e) {
-      throw new IOException("Failed to get numRows from HiveTable", e);
+    } catch (Exception e) {
+      throw new IOException("Failed to get number of rows and total size from HiveTable", e);
     } finally {
       logger.debug("Took {} µs to get stats from {}.{}", timeGetStats.elapsed(TimeUnit.NANOSECONDS) / 1000,
           table.getDbName(), table.getTableName());
@@ -130,7 +132,7 @@ public HiveStats getStats(final HiveReadEntry hiveReadEntry) throws IOException
   }
 
   /** Helper method which return InputSplits for non-partitioned table */
-  private List<LogicalInputSplit> getTableInputSplits() throws Exception {
+  private List<LogicalInputSplit> getTableInputSplits() {
     Preconditions.checkState(!isPartitionedTable, "Works only for non-partitioned tables");
     if (tableInputSplits != null) {
       return tableInputSplits;
@@ -145,7 +147,7 @@ public HiveStats getStats(final HiveReadEntry hiveReadEntry) throws IOException
   /** Helper method which returns the InputSplits for given partition. InputSplits are cached to speed up subsequent
    * metadata cache requests for the same partition(s).
    */
-  private List<LogicalInputSplit> getPartitionInputSplits(final HivePartition partition) throws Exception {
+  private List<LogicalInputSplit> getPartitionInputSplits(final HivePartition partition) {
     if (partitionInputSplitMap.containsKey(partition)) {
       return partitionInputSplitMap.get(partition);
     }
@@ -164,18 +166,17 @@ public HiveStats getStats(final HiveReadEntry hiveReadEntry) throws IOException
    * @param hiveReadEntry Subset of the {@link HiveReadEntry} used when creating this object.
    * @return list of logically grouped input splits
    */
-  public List<LogicalInputSplit> getInputSplits(final HiveReadEntry hiveReadEntry) {
-    final Stopwatch timeGetSplits = Stopwatch.createStarted();
+  public List<LogicalInputSplit> getInputSplits(HiveReadEntry hiveReadEntry) {
+    Stopwatch timeGetSplits = Stopwatch.createStarted();
     try {
       if (!isPartitionedTable) {
         return getTableInputSplits();
       }
 
-      final List<LogicalInputSplit> splits = Lists.newArrayList();
-      for (HivePartition p : hiveReadEntry.getPartitions()) {
-        splits.addAll(getPartitionInputSplits(p));
-      }
-      return splits;
+      return hiveReadEntry.getPartitions().stream()
+          .flatMap(p -> getPartitionInputSplits(p).stream())
+          .collect(Collectors.toList());
+
     } catch (final Exception e) {
       logger.error("Failed to get InputSplits", e);
       throw new DrillRuntimeException("Failed to get InputSplits", e);
@@ -190,63 +191,44 @@ public HiveStats getStats(final HiveReadEntry hiveReadEntry) throws IOException
    *
    * @param hiveReadEntry {@link HiveReadEntry} containing the input table and/or partitions.
    */
-  protected List<String> getInputDirectories(final HiveReadEntry hiveReadEntry) {
+  protected List<String> getInputDirectories(HiveReadEntry hiveReadEntry) {
     if (isPartitionedTable) {
-      final List<String> inputs = Lists.newArrayList();
-      for(Partition p : hiveReadEntry.getPartitions()) {
-        inputs.add(p.getSd().getLocation());
-      }
-      return inputs;
+      return hiveReadEntry.getPartitions().stream()
+          .map(p -> p.getSd().getLocation())
+          .collect(Collectors.toList());
     }
 
     return Collections.singletonList(hiveReadEntry.getTable().getSd().getLocation());
   }
 
   /**
-   * Get the stats from table properties. If not found -1 is returned for each stats field.
-   * CAUTION: stats may not be up-to-date with the underlying data. It is always good to run the ANALYZE command on
-   * Hive table to have up-to-date stats.
+   * Estimate the stats from the given list of logically grouped input splits.
    *
-   * @param properties the source of table stats
-   * @return {@link HiveStats} instance with rows number and size in bytes from specified properties
+   * @param inputSplits list of logically grouped input splits
+   * @return hive stats with numRows and totalSizeInBytes
    */
-  private HiveStats getStatsFromProps(final Properties properties) {
-    long numRows = -1;
-    long sizeInBytes = -1;
-    try {
-      final String numRowsProp = properties.getProperty(StatsSetupConst.ROW_COUNT);
-      if (numRowsProp != null) {
-          numRows = Long.valueOf(numRowsProp);
-      }
-
-      final String sizeInBytesProp = properties.getProperty(StatsSetupConst.TOTAL_SIZE);
-      if (sizeInBytesProp != null) {
-        sizeInBytes = Long.valueOf(sizeInBytesProp);
-      }
-    } catch (final NumberFormatException e) {
-      logger.error("Failed to parse Hive stats in metastore.", e);
-      // continue with the defaults.
+  private HiveStats estimateStatsFromInputSplits(List<LogicalInputSplit> inputSplits) throws IOException {
+    logger.trace("Collecting stats based on input splits size. " +
+        "It means that we might have fetched all input splits before applying any possible optimizations (ex: partition pruning). " +
+        "Consider using ANALYZE command on Hive table to collect statistics before running queries.");
+    long sizeInBytes = 0;
+    for (LogicalInputSplit split : inputSplits) {
+      sizeInBytes += split.getLength();
     }
-
-    return new HiveStats(numRows, sizeInBytes);
+    return estimateStatsFromBytes(sizeInBytes);
   }
 
   /**
-   * Estimate the stats from the given list of logically grouped input splits.
+   * Estimates Hive stats based on give size in bytes.
    *
-   * @param inputSplits list of logically grouped input splits
-   * @return hive stats usually numRows and totalSizeInBytes which used
+   * @param sizeInBytes size in bytes
+   * @return hive stats with numRows and totalSizeInBytes
    */
-  private HiveStats getStatsEstimateFromInputSplits(final List<LogicalInputSplit> inputSplits) throws IOException {
-    long data = 0;
-    for (final LogicalInputSplit split : inputSplits) {
-      data += split.getLength();
-    }
-
-    long numRows = data / RECORD_SIZE;
+  private HiveStats estimateStatsFromBytes(long sizeInBytes) {
+    long numRows = sizeInBytes / RECORD_SIZE;
     // if the result of division is zero and data size > 0, estimate to one row
-    numRows = numRows == 0 && data > 0 ? 1 : numRows;
-    return new HiveStats(numRows, data);
+    numRows = numRows == 0 && sizeInBytes > 0 ? 1 : numRows;
+    return new HiveStats(numRows, sizeInBytes);
   }
 
   /**
@@ -262,36 +244,34 @@ private HiveStats getStatsEstimateFromInputSplits(final List<LogicalInputSplit>
   private List<LogicalInputSplit> splitInputWithUGI(final Properties properties, final StorageDescriptor sd, final Partition partition) {
     watch.start();
     try {
-      return ugi.doAs(new PrivilegedExceptionAction<List<LogicalInputSplit>>() {
-        public List<LogicalInputSplit> run() throws Exception {
-          final List<LogicalInputSplit> splits = Lists.newArrayList();
-          final JobConf job = new JobConf(hiveConf);
-          HiveUtilities.addConfToJob(job, properties);
-          HiveUtilities.verifyAndAddTransactionalProperties(job, sd);
-          job.setInputFormat(HiveUtilities.getInputFormatClass(job, sd, hiveReadEntry.getTable()));
-          final Path path = new Path(sd.getLocation());
-          final FileSystem fs = path.getFileSystem(job);
-          if (fs.exists(path)) {
-            FileInputFormat.addInputPath(job, path);
-            final InputFormat<?, ?> format = job.getInputFormat();
-            InputSplit[] inputSplits = format.getSplits(job, 1);
-
-            // if current table with text input format and has header / footer,
-            // we need to make sure that splits of the same file are grouped together
-            if (TextInputFormat.class.getCanonicalName().equals(sd.getInputFormat()) &&
-                HiveUtilities.hasHeaderOrFooter(hiveReadEntry.getTable())) {
-              Multimap<Path, FileSplit> inputSplitMultimap = transformFileSplits(inputSplits);
-              for (Collection<FileSplit> logicalInputSplit : inputSplitMultimap.asMap().values()) {
-                splits.add(new LogicalInputSplit(logicalInputSplit, partition));
-              }
-            } else {
-              for (final InputSplit split : inputSplits) {
-                splits.add(new LogicalInputSplit(split, partition));
-              }
+      return ugi.doAs((PrivilegedExceptionAction<List<LogicalInputSplit>>) () -> {
+        final List<LogicalInputSplit> splits = new ArrayList<>();
+        final JobConf job = new JobConf(hiveConf);
+        HiveUtilities.addConfToJob(job, properties);
+        HiveUtilities.verifyAndAddTransactionalProperties(job, sd);
+        job.setInputFormat(HiveUtilities.getInputFormatClass(job, sd, hiveReadEntry.getTable()));
+        final Path path = new Path(sd.getLocation());
+        final FileSystem fs = path.getFileSystem(job);
+        if (fs.exists(path)) {
+          FileInputFormat.addInputPath(job, path);
+          final InputFormat<?, ?> format = job.getInputFormat();
+          InputSplit[] inputSplits = format.getSplits(job, 1);
+
+          // if current table with text input format and has header / footer,
+          // we need to make sure that splits of the same file are grouped together
+          if (TextInputFormat.class.getCanonicalName().equals(sd.getInputFormat()) &&
+              HiveUtilities.hasHeaderOrFooter(hiveReadEntry.getTable())) {
+            Multimap<Path, FileSplit> inputSplitMultimap = transformFileSplits(inputSplits);
+            for (Collection<FileSplit> logicalInputSplit : inputSplitMultimap.asMap().values()) {
+              splits.add(new LogicalInputSplit(logicalInputSplit, partition));
+            }
+          } else {
+            for (final InputSplit split : inputSplits) {
+              splits.add(new LogicalInputSplit(split, partition));
             }
           }
-          return splits;
         }
+        return splits;
       });
     } catch (final InterruptedException | IOException e) {
       final String errMsg = String.format("Failed to create input splits: %s", e.getMessage());
@@ -320,13 +300,8 @@ private HiveStats getStatsEstimateFromInputSplits(final List<LogicalInputSplit>
    * @return multimap where key is file path and value is group of ordered file splits
    */
   private Multimap<Path, FileSplit> transformFileSplits(InputSplit[] inputSplits) {
-    Multimap<Path, FileSplit> inputSplitGroups = TreeMultimap.create(Ordering.<Path>natural(),
-        new Comparator<FileSplit>() {
-      @Override
-      public int compare(FileSplit f1, FileSplit f2) {
-        return Long.compare(f1.getStart(), f2.getStart());
-      }
-    });
+    Multimap<Path, FileSplit> inputSplitGroups = TreeMultimap.create(
+        Ordering.natural(), Comparator.comparingLong(FileSplit::getStart));
 
     for (InputSplit inputSplit : inputSplits) {
       FileSplit fileSplit = (FileSplit) inputSplit;
@@ -375,7 +350,7 @@ public long getLength() throws IOException {
     }
 
     /**
-     * @return collection of unique locations where inout splits are stored
+     * @return collection of unique locations where input splits are stored
      */
     public Collection<String> getLocations() throws IOException {
       Set<String> locations = new HashSet<>();
@@ -413,16 +388,53 @@ public String getType() {
     }
   }
 
-  /** Contains stats. Currently only numRows and totalSizeInBytes are used. */
+  /**
+   * Contains stats. Currently only numRows and totalSizeInBytes are used.
+   */
   public static class HiveStats {
+
+    private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HiveStats.class);
+
     private long numRows;
     private long sizeInBytes;
 
-    public HiveStats(final long numRows, final long sizeInBytes) {
+    public HiveStats(long numRows, long sizeInBytes) {
       this.numRows = numRows;
       this.sizeInBytes = sizeInBytes;
     }
 
+    /**
+     * Get the stats from table properties. If not found -1 is returned for each stats field.
+     * CAUTION: stats may not be up-to-date with the underlying data. It is always good to run the ANALYZE command on
+     * Hive table to have up-to-date stats.
+     *
+     * @param properties the source of table stats
+     * @return {@link HiveStats} instance with rows number and size in bytes from specified properties
+     */
+    public static HiveStats getStatsFromProps(Properties properties) {
+      long numRows = -1;
+      long sizeInBytes = -1;
+      try {
+        String sizeInBytesProp = properties.getProperty(StatsSetupConst.TOTAL_SIZE);
+        if (sizeInBytesProp != null) {
+          sizeInBytes = Long.valueOf(sizeInBytesProp);
+        }
+
+        String numRowsProp = properties.getProperty(StatsSetupConst.ROW_COUNT);
+        if (numRowsProp != null) {
+          numRows = Long.valueOf(numRowsProp);
+        }
+      } catch (NumberFormatException e) {
+        logger.error("Failed to parse Hive stats from metastore.", e);
+        // continue with the defaults.
+      }
+
+      HiveStats hiveStats = new HiveStats(numRows, sizeInBytes);
+      logger.trace("Obtained Hive stats from properties: {}.", hiveStats);
+      return hiveStats;
+    }
+
+
     public long getNumRows() {
       return numRows;
     }
@@ -431,7 +443,9 @@ public long getSizeInBytes() {
       return sizeInBytes;
     }
 
-    /** Both numRows and sizeInBytes are expected to be greater than 0 for stats to be valid */
+    /**
+     * Both numRows and sizeInBytes are expected to be greater than 0 for stats to be valid
+     */
     public boolean valid() {
       return numRows > 0 && sizeInBytes > 0;
     }
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveScan.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveScan.java
index 11d47f304c..d631740155 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveScan.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveScan.java
@@ -50,8 +50,6 @@
 import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
 
 import static org.apache.drill.exec.store.hive.DrillHiveMetaStoreClient.createPartitionWithSpecColumns;
 
@@ -64,6 +62,7 @@
   private final HiveStoragePlugin hiveStoragePlugin;
   private final HiveReadEntry hiveReadEntry;
   private final HiveMetadataProvider metadataProvider;
+  private final Map<String, String> confProperties;
 
   private List<List<LogicalInputSplit>> mappings;
   private List<LogicalInputSplit> inputSplits;
@@ -75,22 +74,24 @@ public HiveScan(@JsonProperty("userName") final String userName,
                   @JsonProperty("hiveReadEntry") final HiveReadEntry hiveReadEntry,
                   @JsonProperty("hiveStoragePluginConfig") final HiveStoragePluginConfig hiveStoragePluginConfig,
                   @JsonProperty("columns") final List<SchemaPath> columns,
+                  @JsonProperty("confProperties") final Map<String, String> confProperties,
                   @JacksonInject final StoragePluginRegistry pluginRegistry) throws ExecutionSetupException {
     this(userName,
         hiveReadEntry,
         (HiveStoragePlugin) pluginRegistry.getPlugin(hiveStoragePluginConfig),
         columns,
-        null);
+        null, confProperties);
   }
 
   public HiveScan(final String userName, final HiveReadEntry hiveReadEntry, final HiveStoragePlugin hiveStoragePlugin,
-      final List<SchemaPath> columns, final HiveMetadataProvider metadataProvider) throws ExecutionSetupException {
+                  final List<SchemaPath> columns, final HiveMetadataProvider metadataProvider, final Map<String, String> confProperties) throws ExecutionSetupException {
     super(userName);
     this.hiveReadEntry = hiveReadEntry;
     this.columns = columns;
     this.hiveStoragePlugin = hiveStoragePlugin;
+    this.confProperties = confProperties;
     if (metadataProvider == null) {
-      this.metadataProvider = new HiveMetadataProvider(userName, hiveReadEntry, hiveStoragePlugin.getHiveConf());
+      this.metadataProvider = new HiveMetadataProvider(userName, hiveReadEntry, getHiveConf());
     } else {
       this.metadataProvider = metadataProvider;
     }
@@ -102,10 +103,11 @@ public HiveScan(final HiveScan that) {
     this.hiveReadEntry = that.hiveReadEntry;
     this.hiveStoragePlugin = that.hiveStoragePlugin;
     this.metadataProvider = that.metadataProvider;
+    this.confProperties = that.confProperties;
   }
 
   public HiveScan clone(final HiveReadEntry hiveReadEntry) throws ExecutionSetupException {
-    return new HiveScan(getUserName(), hiveReadEntry, hiveStoragePlugin, columns, metadataProvider);
+    return new HiveScan(getUserName(), hiveReadEntry, hiveStoragePlugin, columns, metadataProvider, confProperties);
   }
 
   @JsonProperty
@@ -123,29 +125,37 @@ public HiveStoragePluginConfig getHiveStoragePluginConfig() {
     return columns;
   }
 
+  @JsonProperty
+  public Map<String, String> getConfProperties() {
+    return confProperties;
+  }
+
   @JsonIgnore
   public HiveStoragePlugin getStoragePlugin() {
     return hiveStoragePlugin;
   }
 
-  protected HiveMetadataProvider getMetadataProvider() {
-    return metadataProvider;
+  @JsonIgnore
+  public HiveConf getHiveConf() {
+    return HiveUtilities.generateHiveConf(hiveStoragePlugin.getHiveConf(), confProperties);
   }
 
-  private List<LogicalInputSplit> getInputSplits() {
-    if (inputSplits == null) {
-      inputSplits = metadataProvider.getInputSplits(hiveReadEntry);
-    }
-
-    return inputSplits;
+  @JsonIgnore
+  public boolean isNativeReader() {
+    return false;
   }
 
+  @Override
+  public boolean supportsPartitionFilterPushdown() {
+    List<FieldSchema> partitionKeys = hiveReadEntry.getTable().getPartitionKeys();
+    return !(partitionKeys == null || partitionKeys.size() == 0);
+  }
 
   @Override
   public void applyAssignments(final List<CoordinationProtos.DrillbitEndpoint> endpoints) {
     mappings = new ArrayList<>();
     for (int i = 0; i < endpoints.size(); i++) {
-      mappings.add(new ArrayList<LogicalInputSplit>());
+      mappings.add(new ArrayList<>());
     }
     final int count = endpoints.size();
     final List<LogicalInputSplit> inputSplits = getInputSplits();
@@ -158,9 +168,9 @@ public void applyAssignments(final List<CoordinationProtos.DrillbitEndpoint> end
   public SubScan getSpecificScan(final int minorFragmentId) throws ExecutionSetupException {
     try {
       final List<LogicalInputSplit> splits = mappings.get(minorFragmentId);
-      List<HivePartitionWrapper> parts = Lists.newArrayList();
-      final List<List<String>> encodedInputSplits = Lists.newArrayList();
-      final List<String> splitTypes = Lists.newArrayList();
+      List<HivePartitionWrapper> parts = new ArrayList<>();
+      final List<List<String>> encodedInputSplits = new ArrayList<>();
+      final List<String> splitTypes = new ArrayList<>();
       for (final LogicalInputSplit split : splits) {
         final Partition splitPartition = split.getPartition();
         if (splitPartition != null) {
@@ -176,7 +186,7 @@ public SubScan getSpecificScan(final int minorFragmentId) throws ExecutionSetupE
       }
 
       final HiveReadEntry subEntry = new HiveReadEntry(hiveReadEntry.getTableWrapper(), parts);
-      return new HiveSubScan(getUserName(), encodedInputSplits, subEntry, splitTypes, columns, hiveStoragePlugin);
+      return new HiveSubScan(getUserName(), encodedInputSplits, subEntry, splitTypes, columns, hiveStoragePlugin, confProperties);
     } catch (IOException | ReflectiveOperationException e) {
       throw new ExecutionSetupException(e);
     }
@@ -192,7 +202,7 @@ public int getMaxParallelizationWidth() {
     final Map<String, DrillbitEndpoint> endpointMap = new HashMap<>();
     for (final DrillbitEndpoint endpoint : hiveStoragePlugin.getContext().getBits()) {
       endpointMap.put(endpoint.getAddress(), endpoint);
-      logger.debug("endpoing address: {}", endpoint.getAddress());
+      logger.debug("Endpoint address: {}", endpoint.getAddress());
     }
     final Map<DrillbitEndpoint, EndpointAffinity> affinityMap = new HashMap<>();
     try {
@@ -204,7 +214,7 @@ public int getMaxParallelizationWidth() {
       for (final LogicalInputSplit split : inputSplits) {
         final float affinity = ((float) Math.max(1, split.getLength())) / totalSize;
         for (final String loc : split.getLocations()) {
-          logger.debug("split location: {}", loc);
+          logger.debug("Split location: {}", loc);
           final DrillbitEndpoint endpoint = endpointMap.get(loc);
           if (endpoint != null) {
             if (affinityMap.containsKey(endpoint)) {
@@ -218,13 +228,8 @@ public int getMaxParallelizationWidth() {
     } catch (final IOException e) {
       throw new DrillRuntimeException(e);
     }
-    for (final DrillbitEndpoint ep : affinityMap.keySet()) {
-      Preconditions.checkNotNull(ep);
-    }
-    for (final EndpointAffinity a : affinityMap.values()) {
-      Preconditions.checkNotNull(a.getEndpoint());
-    }
-    return Lists.newArrayList(affinityMap.values());
+
+    return new ArrayList<>(affinityMap.values());
   }
 
   @Override
@@ -243,21 +248,8 @@ public ScanStats getScanStats() {
     }
   }
 
-  protected int getSerDeOverheadFactor() {
-    final int projectedColumnCount;
-    if (Utilities.isStarQuery(columns)) {
-      Table hiveTable = hiveReadEntry.getTable();
-      projectedColumnCount = hiveTable.getSd().getColsSize() + hiveTable.getPartitionKeysSize();
-    } else {
-      // In cost estimation, # of project columns should be >= 1, even for skipAll query.
-      projectedColumnCount = Math.max(columns.size(), 1);
-    }
-
-    return projectedColumnCount * HIVE_SERDE_SCAN_OVERHEAD_FACTOR_PER_COLUMN;
-  }
-
   @Override
-  public PhysicalOperator getNewWithChildren(final List<PhysicalOperator> children) throws ExecutionSetupException {
+  public PhysicalOperator getNewWithChildren(final List<PhysicalOperator> children) {
     return new HiveScan(this);
   }
 
@@ -275,6 +267,7 @@ public String toString() {
         + ", numPartitions=" + numPartitions
         + ", partitions= " + partitions
         + ", inputDirectories=" + metadataProvider.getInputDirectories(hiveReadEntry)
+        + ", confProperties=" + confProperties
         + "]";
   }
 
@@ -290,22 +283,24 @@ public boolean canPushdownProjects(final List<SchemaPath> columns) {
     return true;
   }
 
-  // Return true if the current table is partitioned false otherwise
-  public boolean supportsPartitionFilterPushdown() {
-    final List<FieldSchema> partitionKeys = hiveReadEntry.getTable().getPartitionKeys();
-    if (partitionKeys == null || partitionKeys.size() == 0) {
-      return false;
+  private List<LogicalInputSplit> getInputSplits() {
+    if (inputSplits == null) {
+      inputSplits = metadataProvider.getInputSplits(hiveReadEntry);
     }
-    return true;
-  }
 
-  @JsonIgnore
-  public HiveConf getHiveConf() {
-    return hiveStoragePlugin.getHiveConf();
+    return inputSplits;
   }
 
-  @JsonIgnore
-  public boolean isNativeReader() {
-    return false;
+  private int getSerDeOverheadFactor() {
+    final int projectedColumnCount;
+    if (Utilities.isStarQuery(columns)) {
+      Table hiveTable = hiveReadEntry.getTable();
+      projectedColumnCount = hiveTable.getSd().getColsSize() + hiveTable.getPartitionKeysSize();
+    } else {
+      // In cost estimation, # of project columns should be >= 1, even for skipAll query.
+      projectedColumnCount = Math.max(columns.size(), 1);
+    }
+
+    return projectedColumnCount * HIVE_SERDE_SCAN_OVERHEAD_FACTOR_PER_COLUMN;
   }
 }
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveStoragePlugin.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveStoragePlugin.java
index 449a6f90b3..adf134843a 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveStoragePlugin.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveStoragePlugin.java
@@ -18,25 +18,34 @@
 package org.apache.drill.exec.store.hive;
 
 import java.io.IOException;
+import java.io.StringReader;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Map.Entry;
+import java.util.Properties;
 import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
 
 import com.google.common.collect.ImmutableSet;
 
 import org.apache.calcite.schema.Schema.TableType;
 import org.apache.calcite.schema.SchemaPlus;
 
+import org.apache.commons.lang3.StringEscapeUtils;
 import org.apache.drill.common.JSONOptions;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.ops.OptimizerRulesContext;
+import org.apache.drill.exec.physical.base.AbstractGroupScan;
+import org.apache.drill.exec.planner.sql.logical.ConvertHiveMapRDBJsonScanToDrillMapRDBJsonScan;
 import org.apache.drill.exec.planner.sql.logical.ConvertHiveParquetScanToDrillParquetScan;
 import org.apache.drill.exec.planner.sql.logical.HivePushPartitionFilterIntoScan;
 import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.server.options.SessionOptionManager;
 import org.apache.drill.exec.store.AbstractStoragePlugin;
 import org.apache.drill.exec.store.SchemaConfig;
 import org.apache.drill.exec.store.StoragePluginOptimizerRule;
@@ -51,17 +60,16 @@
 
 public class HiveStoragePlugin extends AbstractStoragePlugin {
 
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HiveStoragePlugin.class);
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HiveStoragePlugin.class);
 
   private final HiveStoragePluginConfig config;
   private HiveSchemaFactory schemaFactory;
   private final HiveConf hiveConf;
 
-  public HiveStoragePlugin(HiveStoragePluginConfig config, DrillbitContext context, String name)
-      throws ExecutionSetupException {
+  public HiveStoragePlugin(HiveStoragePluginConfig config, DrillbitContext context, String name) throws ExecutionSetupException {
     super(context, name);
     this.config = config;
-    this.hiveConf = createHiveConf(config.getHiveConfigOverride());
+    this.hiveConf = HiveUtilities.generateHiveConf(config.getConfigProps());
     this.schemaFactory = new HiveSchemaFactory(this, name, hiveConf);
   }
 
@@ -73,8 +81,18 @@ public HiveStoragePluginConfig getConfig() {
     return config;
   }
 
+  @Override
+  public HiveScan getPhysicalScan(String userName, JSONOptions selection, SessionOptionManager options) throws IOException {
+    return getPhysicalScan(userName, selection, AbstractGroupScan.ALL_COLUMNS, options);
+  }
+
   @Override
   public HiveScan getPhysicalScan(String userName, JSONOptions selection, List<SchemaPath> columns) throws IOException {
+    return getPhysicalScan(userName, selection, columns, null);
+  }
+
+  @Override
+  public HiveScan getPhysicalScan(String userName, JSONOptions selection, List<SchemaPath> columns, SessionOptionManager options) throws IOException {
     HiveReadEntry hiveReadEntry = selection.getListWith(new ObjectMapper(), new TypeReference<HiveReadEntry>(){});
     try {
       if (hiveReadEntry.getJdbcTableType() == TableType.VIEW) {
@@ -82,7 +100,26 @@ public HiveScan getPhysicalScan(String userName, JSONOptions selection, List<Sch
             "Querying views created in Hive from Drill is not supported in current version.");
       }
 
-      return new HiveScan(userName, hiveReadEntry, this, columns, null);
+      Map<String, String> confProperties = new HashMap<>();
+      if (options != null) {
+        String value = StringEscapeUtils.unescapeJava(options.getString(ExecConstants.HIVE_CONF_PROPERTIES));
+        logger.trace("[{}] is set to {}.", ExecConstants.HIVE_CONF_PROPERTIES, value);
+        try {
+          Properties properties = new Properties();
+          properties.load(new StringReader(value));
+          confProperties =
+            properties.stringPropertyNames().stream()
+              .collect(
+                Collectors.toMap(
+                  Function.identity(),
+                  properties::getProperty,
+                  (o, n) -> n));
+          } catch (IOException e) {
+            logger.warn("Unable to parse Hive conf properties {}, ignoring them.", value);
+        }
+      }
+
+      return new HiveScan(userName, hiveReadEntry, this, columns, null, confProperties);
     } catch (ExecutionSetupException e) {
       throw new IOException(e);
     }
@@ -165,24 +202,18 @@ public synchronized void registerSchemas(SchemaConfig schemaConfig, SchemaPlus p
 
   @Override
   public Set<StoragePluginOptimizerRule> getPhysicalOptimizerRules(OptimizerRulesContext optimizerRulesContext) {
+    ImmutableSet.Builder<StoragePluginOptimizerRule> ruleBuilder = ImmutableSet.builder();
+    OptionManager options = optimizerRulesContext.getPlannerSettings().getOptions();
     // TODO: Remove implicit using of convert_fromTIMESTAMP_IMPALA function
     // once "store.parquet.reader.int96_as_timestamp" will be true by default
-    if(optimizerRulesContext.getPlannerSettings().getOptions()
-        .getOption(ExecConstants.HIVE_OPTIMIZE_SCAN_WITH_NATIVE_READERS).bool_val) {
-      return ImmutableSet.<StoragePluginOptimizerRule>of(ConvertHiveParquetScanToDrillParquetScan.INSTANCE);
+    if (options.getBoolean(ExecConstants.HIVE_OPTIMIZE_SCAN_WITH_NATIVE_READERS) ||
+        options.getBoolean(ExecConstants.HIVE_OPTIMIZE_PARQUET_SCAN_WITH_NATIVE_READER)) {
+      ruleBuilder.add(ConvertHiveParquetScanToDrillParquetScan.INSTANCE);
     }
-
-    return ImmutableSet.of();
-  }
-
-  private static HiveConf createHiveConf(final Map<String, String> hiveConfigOverride) {
-    final HiveConf hiveConf = new HiveConf();
-    for(Entry<String, String> config : hiveConfigOverride.entrySet()) {
-      final String key = config.getKey();
-      final String value = config.getValue();
-      hiveConf.set(key, value);
-      logger.trace("HiveConfig Override {}={}", key, value);
+    if (options.getBoolean(ExecConstants.HIVE_OPTIMIZE_MAPRDB_JSON_SCAN_WITH_NATIVE_READER)) {
+      ruleBuilder.add(ConvertHiveMapRDBJsonScanToDrillMapRDBJsonScan.INSTANCE);
     }
-    return hiveConf;
+    return ruleBuilder.build();
   }
+
 }
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveStoragePluginConfig.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveStoragePluginConfig.java
index b6f15c81e3..d812468884 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveStoragePluginConfig.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveStoragePluginConfig.java
@@ -19,28 +19,31 @@
 
 import java.util.Map;
 
+import com.fasterxml.jackson.annotation.JsonAlias;
 import org.apache.drill.common.logical.StoragePluginConfigBase;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
 
 @JsonTypeName(HiveStoragePluginConfig.NAME)
 public class HiveStoragePluginConfig extends StoragePluginConfigBase {
-  @JsonProperty
-  public Map<String, String> configProps;
 
   public static final String NAME = "hive";
 
-  @JsonIgnore
-  public Map<String, String> getHiveConfigOverride() {
-    return configProps;
-  }
+  private final Map<String, String> configProps;
 
   @JsonCreator
-  public HiveStoragePluginConfig(@JsonProperty("config") Map<String, String> props) {
-    this.configProps = props;
+  public HiveStoragePluginConfig(@JsonProperty("configProps")
+                                 // previously two names were allowed due to incorrectly written ser / der logic
+                                 // allowing to use both during deserialization for backward compatibility
+                                 @JsonAlias("config") Map<String, String> configProps) {
+    this.configProps = configProps;
+  }
+
+  @JsonProperty
+  public Map<String, String> getConfigProps() {
+    return configProps;
   }
 
   @Override
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveSubScan.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveSubScan.java
index 8ca8647cb1..0acec2db4b 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveSubScan.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveSubScan.java
@@ -22,6 +22,7 @@
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 
 import com.fasterxml.jackson.annotation.JacksonInject;
 import com.google.common.collect.ImmutableSet;
@@ -55,6 +56,7 @@
   private final HiveTableWithColumnCache table;
   private final List<HivePartition> partitions;
   private final List<SchemaPath> columns;
+  private final Map<String, String> confProperties;
 
   @JsonCreator
   public HiveSubScan(@JacksonInject StoragePluginRegistry registry,
@@ -63,22 +65,24 @@ public HiveSubScan(@JacksonInject StoragePluginRegistry registry,
                      @JsonProperty("hiveReadEntry") HiveReadEntry hiveReadEntry,
                      @JsonProperty("splitClasses") List<String> splitClasses,
                      @JsonProperty("columns") List<SchemaPath> columns,
-                     @JsonProperty("hiveStoragePluginConfig") HiveStoragePluginConfig hiveStoragePluginConfig)
+                     @JsonProperty("hiveStoragePluginConfig") HiveStoragePluginConfig hiveStoragePluginConfig,
+                     @JsonProperty("confProperties") Map<String, String> confProperties)
       throws IOException, ExecutionSetupException, ReflectiveOperationException {
     this(userName,
         splits,
         hiveReadEntry,
         splitClasses,
         columns,
-        (HiveStoragePlugin) registry.getPlugin(hiveStoragePluginConfig));
+        (HiveStoragePlugin) registry.getPlugin(hiveStoragePluginConfig), confProperties);
   }
 
   public HiveSubScan(final String userName,
                      final List<List<String>> splits,
                      final HiveReadEntry hiveReadEntry,
-                      final List<String> splitClasses,
+                     final List<String> splitClasses,
                      final List<SchemaPath> columns,
-                     final HiveStoragePlugin hiveStoragePlugin)
+                     final HiveStoragePlugin hiveStoragePlugin,
+                     final Map<String, String> confProperties)
     throws IOException, ReflectiveOperationException {
     super(userName);
     this.hiveReadEntry = hiveReadEntry;
@@ -88,6 +92,7 @@ public HiveSubScan(final String userName,
     this.splitClasses = splitClasses;
     this.columns = columns;
     this.hiveStoragePlugin = hiveStoragePlugin;
+    this.confProperties = confProperties;
 
     for (int i = 0; i < splits.size(); i++) {
       inputSplits.add(deserializeInputSplit(splits.get(i), splitClasses.get(i)));
@@ -119,6 +124,11 @@ public HiveStoragePluginConfig getHiveStoragePluginConfig() {
     return hiveStoragePlugin.getConfig();
   }
 
+  @JsonProperty
+  public Map<String, String> getConfProperties() {
+    return confProperties;
+  }
+
   @JsonIgnore
   public HiveTableWithColumnCache getTable() {
     return table;
@@ -141,7 +151,7 @@ public HiveStoragePlugin getStoragePlugin() {
 
   @JsonIgnore
   public HiveConf getHiveConf() {
-    return hiveStoragePlugin.getHiveConf();
+    return HiveUtilities.generateHiveConf(hiveStoragePlugin.getHiveConf(), confProperties);
   }
 
   @Override
@@ -152,7 +162,7 @@ public HiveConf getHiveConf() {
   @Override
   public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) throws ExecutionSetupException {
     try {
-      return new HiveSubScan(getUserName(), splits, hiveReadEntry, splitClasses, columns, hiveStoragePlugin);
+      return new HiveSubScan(getUserName(), splits, hiveReadEntry, splitClasses, columns, hiveStoragePlugin, confProperties);
     } catch (IOException | ReflectiveOperationException e) {
       throw new ExecutionSetupException(e);
     }
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveTableWrapper.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveTableWrapper.java
index f5ebc5db94..e6178b2a4a 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveTableWrapper.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveTableWrapper.java
@@ -55,7 +55,7 @@
   @JsonProperty
   public List<FieldSchemaWrapper> partitionKeys;
   @JsonProperty
-  public Map<String,String> parameters;
+  public Map<String, String> parameters;
   @JsonProperty
   public String viewOriginalText;
   @JsonProperty
@@ -129,6 +129,11 @@ public HiveTableWithColumnCache getTable() {
     return table;
   }
 
+  @JsonIgnore
+  public Map<String, String> getParameters() {
+    return parameters;
+  }
+
   @Override
   public String toString() {
     StringBuilder sb = new StringBuilder("Table(");
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveUtilities.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveUtilities.java
index 5279f281d1..6fc567ee19 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveUtilities.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveUtilities.java
@@ -17,12 +17,10 @@
  */
 package org.apache.drill.exec.store.hive;
 
-import com.google.common.base.Function;
-import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
-import com.google.common.collect.Lists;
 import io.netty.buffer.DrillBuf;
+import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.exceptions.UserException;
@@ -34,6 +32,7 @@
 import org.apache.drill.exec.expr.holders.Decimal28SparseHolder;
 import org.apache.drill.exec.expr.holders.Decimal38SparseHolder;
 import org.apache.drill.exec.expr.holders.Decimal9Holder;
+import org.apache.drill.exec.planner.logical.DrillScanRel;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.server.options.OptionSet;
 import org.apache.drill.exec.util.DecimalUtility;
@@ -60,6 +59,7 @@
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.io.IOConstants;
 import org.apache.hadoop.hive.ql.metadata.HiveStorageHandler;
@@ -73,25 +73,34 @@
 import org.apache.hadoop.hive.serde2.typeinfo.HiveDecimalUtils;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
 import org.apache.hadoop.mapred.InputFormat;
 import org.apache.hadoop.mapred.JobConf;
 import org.joda.time.DateTime;
 import org.joda.time.DateTimeZone;
 
-import javax.annotation.Nullable;
 import java.math.BigDecimal;
 import java.math.RoundingMode;
 import java.sql.Date;
 import java.sql.Timestamp;
+import java.util.List;
 import java.util.Map;
 import java.util.Properties;
+import java.util.stream.Collectors;
 
 import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_STORAGE;
 
 public class HiveUtilities {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HiveUtilities.class);
 
-  /** Partition value is received in string format. Convert it into appropriate object based on the type. */
+  /**
+   * Partition value is received in string format. Convert it into appropriate object based on the type.
+   *
+   * @param typeInfo type info
+   * @param value partition values
+   * @param defaultPartitionValue default partition value
+   * @return converted object
+   */
   public static Object convertPartitionType(TypeInfo typeInfo, String value, final String defaultPartitionValue) {
     if (typeInfo.getCategory() != Category.PRIMITIVE) {
       // In Hive only primitive types are allowed as partition column types.
@@ -145,6 +154,15 @@ public static Object convertPartitionType(TypeInfo typeInfo, String value, final
     return null;
   }
 
+  /**
+   * Populates vector with given value based on its type.
+   *
+   * @param vector vector instance
+   * @param managedBuffer Drill duffer
+   * @param val value
+   * @param start start position
+   * @param end end position
+   */
   public static void populateVector(final ValueVector vector, final DrillBuf managedBuffer, final Object val,
       final int start, final int end) {
     TypeProtos.MinorType type = vector.getField().getType().getMinorType();
@@ -305,6 +323,13 @@ public static void populateVector(final ValueVector vector, final DrillBuf manag
     }
   }
 
+  /**
+   * Obtains major type from given type info holder.
+   *
+   * @param typeInfo type info holder
+   * @param options session options
+   * @return appropriate major type, null otherwise. For some types may throw unsupported exception.
+   */
   public static MajorType getMajorTypeFromHiveTypeInfo(final TypeInfo typeInfo, final OptionSet options) {
     switch (typeInfo.getCategory()) {
       case PRIMITIVE: {
@@ -341,8 +366,14 @@ public static MajorType getMajorTypeFromHiveTypeInfo(final TypeInfo typeInfo, fi
     return null;
   }
 
-  public static TypeProtos.MinorType getMinorTypeFromHivePrimitiveTypeInfo(PrimitiveTypeInfo primitiveTypeInfo,
-                                                                           OptionSet options) {
+  /**
+   * Obtains minor type from given primitive type info holder.
+   *
+   * @param primitiveTypeInfo primitive type info holder
+   * @param options session options
+   * @return appropriate minor type, otherwise throws unsupported type exception
+   */
+  public static TypeProtos.MinorType getMinorTypeFromHivePrimitiveTypeInfo(PrimitiveTypeInfo primitiveTypeInfo, OptionSet options) {
     switch(primitiveTypeInfo.getPrimitiveCategory()) {
       case BINARY:
         return TypeProtos.MinorType.VARBINARY;
@@ -390,10 +421,8 @@ public static MajorType getMajorTypeFromHiveTypeInfo(final TypeInfo typeInfo, fi
    * @param job {@link JobConf} instance needed incase the table is StorageHandler based table.
    * @param sd {@link StorageDescriptor} instance of currently reading partition or table (for non-partitioned tables).
    * @param table Table object
-   * @throws Exception
    */
-  public static Class<? extends InputFormat<?, ?>> getInputFormatClass(final JobConf job, final StorageDescriptor sd,
-      final Table table) throws Exception {
+  public static Class<? extends InputFormat<?, ?>> getInputFormatClass(final JobConf job, final StorageDescriptor sd, final Table table) throws Exception {
     final String inputFormatName = sd.getInputFormat();
     if (Strings.isNullOrEmpty(inputFormatName)) {
       final String storageHandlerClass = table.getParameters().get(META_TABLE_STORAGE);
@@ -424,26 +453,23 @@ public static void addConfToJob(final JobConf job, final Properties properties)
   }
 
   /**
-   * Wrapper around {@link MetaStoreUtils#getPartitionMetadata(Partition, Table)} which also adds parameters from table
-   * to properties returned by {@link MetaStoreUtils#getPartitionMetadata(Partition, Table)}.
+   * Wrapper around {@link MetaStoreUtils#getPartitionMetadata(org.apache.hadoop.hive.metastore.api.Partition, Table)}
+   * which also adds parameters from table to properties returned by that method.
    *
    * @param partition the source of partition level parameters
    * @param table     the source of table level parameters
    * @return properties
    */
   public static Properties getPartitionMetadata(final HivePartition partition, final HiveTableWithColumnCache table) {
-    final Properties properties;
     restoreColumns(table, partition);
-    properties = MetaStoreUtils.getPartitionMetadata(partition, table);
+    Properties properties = MetaStoreUtils.getPartitionMetadata(partition, table);
 
     // SerDe expects properties from Table, but above call doesn't add Table properties.
     // Include Table properties in final list in order to not to break SerDes that depend on
     // Table properties. For example AvroSerDe gets the schema from properties (passed as second argument)
-    for (Map.Entry<String, String> entry : table.getParameters().entrySet()) {
-      if (entry.getKey() != null && entry.getKey() != null) {
-        properties.put(entry.getKey(), entry.getValue());
-      }
-    }
+    table.getParameters().entrySet().stream()
+        .filter(e -> e.getKey() != null && e.getValue() != null)
+        .forEach(e -> properties.put(e.getKey(), e.getValue()));
 
     return properties;
   }
@@ -451,8 +477,8 @@ public static Properties getPartitionMetadata(final HivePartition partition, fin
   /**
    * Sets columns from table cache to table and partition.
    *
+   * @param table the source of column lists cache
    * @param partition partition which will set column list
-   * @param table     the source of column lists cache
    */
   public static void restoreColumns(HiveTableWithColumnCache table, HivePartition partition) {
     // exactly the same column lists for partitions or table
@@ -469,6 +495,9 @@ public static void restoreColumns(HiveTableWithColumnCache table, HivePartition
    * Wrapper around {@link MetaStoreUtils#getSchema(StorageDescriptor, StorageDescriptor, Map, String, String, List)}
    * which also sets columns from table cache to table and returns properties returned by
    * {@link MetaStoreUtils#getSchema(StorageDescriptor, StorageDescriptor, Map, String, String, List)}.
+   *
+   * @param table Hive table with cached columns
+   * @return Hive table metadata
    */
   public static Properties getTableMetadata(HiveTableWithColumnCache table) {
     restoreColumns(table, null);
@@ -476,13 +505,18 @@ public static Properties getTableMetadata(HiveTableWithColumnCache table) {
       table.getDbName(), table.getTableName(), table.getPartitionKeys());
   }
 
+  /**
+   * Generates unsupported types exception message with list of supported types
+   * and throws user exception.
+   *
+   * @param unsupportedType unsupported type
+   */
   public static void throwUnsupportedHiveDataTypeError(String unsupportedType) {
-    StringBuilder errMsg = new StringBuilder();
-    errMsg.append(String.format("Unsupported Hive data type %s. ", unsupportedType));
-    errMsg.append(System.getProperty("line.separator"));
-    errMsg.append("Following Hive data types are supported in Drill for querying: ");
-    errMsg.append(
-        "BOOLEAN, TINYINT, SMALLINT, INT, BIGINT, FLOAT, DOUBLE, DATE, TIMESTAMP, BINARY, DECIMAL, STRING, VARCHAR and CHAR");
+    StringBuilder errMsg = new StringBuilder()
+        .append("Unsupported Hive data type ").append(unsupportedType).append(". ")
+        .append(System.lineSeparator())
+        .append("Following Hive data types are supported in Drill for querying: ")
+        .append("BOOLEAN, TINYINT, SMALLINT, INT, BIGINT, FLOAT, DOUBLE, DATE, TIMESTAMP, BINARY, DECIMAL, STRING, VARCHAR and CHAR");
 
     throw UserException.unsupportedError()
         .message(errMsg.toString())
@@ -528,7 +562,7 @@ public static boolean hasHeaderOrFooter(HiveTableWithColumnCache table) {
   }
 
   /**
-   * This method checks whether the table is transactional and set necessary properties in {@link JobConf}.
+   * This method checks whether the table is transactional and set necessary properties in {@link JobConf}.<br>
    * If schema evolution properties aren't set in job conf for the input format, method sets the column names
    * and types from table/partition properties or storage descriptor.
    *
@@ -555,30 +589,159 @@ public static void verifyAndAddTransactionalProperties(JobConf job, StorageDescr
       colTypes = job.get(serdeConstants.LIST_COLUMN_TYPES);
 
       if (colNames == null || colTypes == null) {
-        colNames = Joiner.on(",").join(Lists.transform(sd.getCols(), new Function<FieldSchema, String>()
-        {
-          @Nullable
-          @Override
-          public String apply(@Nullable FieldSchema input)
-          {
-            return input.getName();
-          }
-        }));
-
-        colTypes = Joiner.on(",").join(Lists.transform(sd.getCols(), new Function<FieldSchema, String>()
-        {
-          @Nullable
-          @Override
-          public String apply(@Nullable FieldSchema input)
-          {
-            return input.getType();
-          }
-        }));
+        colNames = sd.getCols().stream()
+            .map(FieldSchema::getName)
+            .collect(Collectors.joining(","));
+        colTypes = sd.getCols().stream()
+            .map(FieldSchema::getType)
+            .collect(Collectors.joining(","));
       }
 
       job.set(IOConstants.SCHEMA_EVOLUTION_COLUMNS, colNames);
       job.set(IOConstants.SCHEMA_EVOLUTION_COLUMNS_TYPES, colTypes);
     }
   }
+
+  /**
+   * Rule is matched when all of the following match:
+   * <ul>
+   * <li>GroupScan in given DrillScalRel is an {@link HiveScan}</li>
+   * <li> {@link HiveScan} is not already rewritten using Drill's native readers</li>
+   * <li> InputFormat in table metadata and all partitions metadata contains the same value {@param tableInputFormatClass}</li>
+   * <li> No error occurred while checking for the above conditions. An error is logged as warning.</li>
+   *</ul>
+   * @param call rule call
+   * @return True if the rule can be applied. False otherwise
+   */
+  public static boolean nativeReadersRuleMatches(RelOptRuleCall call, Class tableInputFormatClass) {
+    final DrillScanRel scanRel = call.rel(0);
+
+    if (!(scanRel.getGroupScan() instanceof HiveScan) || ((HiveScan) scanRel.getGroupScan()).isNativeReader()) {
+      return false;
+    }
+
+    final HiveScan hiveScan = (HiveScan) scanRel.getGroupScan();
+    final HiveConf hiveConf = hiveScan.getHiveConf();
+    final HiveTableWithColumnCache hiveTable = hiveScan.getHiveReadEntry().getTable();
+
+    if (HiveUtilities.containsUnsupportedDataTypes(hiveTable)) {
+      return false;
+    }
+
+    final Class<? extends InputFormat<?, ?>> tableInputFormat = getInputFormatFromSD(
+        HiveUtilities.getTableMetadata(hiveTable), hiveScan.getHiveReadEntry(), hiveTable.getSd(), hiveConf);
+    if (tableInputFormat == null || !tableInputFormat.equals(tableInputFormatClass)) {
+      return false;
+    }
+
+    final List<HiveTableWrapper.HivePartitionWrapper> partitions = hiveScan.getHiveReadEntry().getHivePartitionWrappers();
+    if (partitions == null) {
+      return true;
+    }
+
+    final List<FieldSchema> tableSchema = hiveTable.getSd().getCols();
+    // Make sure all partitions have the same input format as the table input format
+    for (HiveTableWrapper.HivePartitionWrapper partition : partitions) {
+      final StorageDescriptor partitionSD = partition.getPartition().getSd();
+      Class<? extends InputFormat<?, ?>> inputFormat = getInputFormatFromSD(HiveUtilities.getPartitionMetadata(
+          partition.getPartition(), hiveTable), hiveScan.getHiveReadEntry(), partitionSD, hiveConf);
+      if (inputFormat == null || !inputFormat.equals(tableInputFormat)) {
+        return false;
+      }
+
+      // Make sure the schema of the table and schema of the partition matches. If not return false. Schema changes
+      // between table and partition can happen when table schema is altered using ALTER statements after some
+      // partitions are already created. Currently native reader conversion doesn't handle schema changes between
+      // partition and table. Hive has extensive list of convert methods to convert from one type to rest of the
+      // possible types. Drill doesn't have the similar set of methods yet.
+      if (!partitionSD.getCols().equals(tableSchema)) {
+        logger.debug("Partitions schema is different from table schema. Currently native reader conversion can't " +
+            "handle schema difference between partitions and table");
+        return false;
+      }
+    }
+
+    return true;
+  }
+
+  /**
+   * Get the input format from given {@link StorageDescriptor}.
+   *
+   * @param properties table properties
+   * @param hiveReadEntry hive read entry
+   * @param sd storage descriptor
+   * @return {@link InputFormat} class or null if a failure has occurred. Failure is logged as warning.
+   */
+  private static Class<? extends InputFormat<?, ?>> getInputFormatFromSD(final Properties properties,
+                                                                  final HiveReadEntry hiveReadEntry, final StorageDescriptor sd, final HiveConf hiveConf) {
+    final Table hiveTable = hiveReadEntry.getTable();
+    try {
+      final String inputFormatName = sd.getInputFormat();
+      if (!Strings.isNullOrEmpty(inputFormatName)) {
+        return (Class<? extends InputFormat<?, ?>>) Class.forName(inputFormatName);
+      }
+
+      final JobConf job = new JobConf(hiveConf);
+      HiveUtilities.addConfToJob(job, properties);
+      return HiveUtilities.getInputFormatClass(job, sd, hiveTable);
+    } catch (final Exception e) {
+      logger.warn("Failed to get InputFormat class from Hive table '{}.{}'. StorageDescriptor [{}]",
+          hiveTable.getDbName(), hiveTable.getTableName(), sd.toString(), e);
+      return null;
+    }
+  }
+
+  /**
+   * This method allows to check whether the Hive Table contains
+   * <a href="https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Types#LanguageManualTypes-ComplexTypes">
+   * Hive Complex Types</a><p>
+   * TODO: Need to implement it, DRILL-3290. Appropriate (new or existed) Drill types should be selected.
+   *
+   * @param hiveTable Thrift table from Hive Metastore
+   * @return true if table contains unsupported data types, false otherwise
+   */
+  public static boolean containsUnsupportedDataTypes(final Table hiveTable) {
+    for (FieldSchema hiveField : hiveTable.getSd().getCols()) {
+      final Category category = TypeInfoUtils.getTypeInfoFromTypeString(hiveField.getType()).getCategory();
+      if (category == Category.MAP ||
+          category == Category.STRUCT ||
+          category == Category.UNION ||
+          category == Category.LIST) {
+        logger.debug("Hive table contains unsupported data type: {}", category);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Creates HiveConf based on given list of configuration properties.
+   *
+   * @param properties config properties
+   * @return instance of HiveConf
+   */
+  public static HiveConf generateHiveConf(Map<String, String> properties) {
+    logger.trace("Override HiveConf with the following properties {}", properties);
+    HiveConf hiveConf = new HiveConf();
+    properties.forEach(hiveConf::set);
+    return hiveConf;
+  }
+
+  /**
+   * Creates HiveConf based on properties in given HiveConf and configuration properties.
+   *
+   * @param hiveConf hive conf
+   * @param properties config properties
+   * @return instance of HiveConf
+   */
+  public static HiveConf generateHiveConf(HiveConf hiveConf, Map<String, String> properties) {
+    Properties changedProperties = hiveConf.getChangedProperties();
+    changedProperties.putAll(properties);
+    HiveConf newHiveConf = new HiveConf();
+    changedProperties.stringPropertyNames()
+        .forEach(name -> newHiveConf.set(name, changedProperties.getProperty(name)));
+    return newHiveConf;
+  }
+
 }
 
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/schema/HiveSchemaFactory.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/schema/HiveSchemaFactory.java
index d3115b8a6e..e3cb3a2dd7 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/schema/HiveSchemaFactory.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/schema/HiveSchemaFactory.java
@@ -27,7 +27,6 @@
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
 import com.google.common.cache.RemovalListener;
-import com.google.common.cache.RemovalNotification;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
@@ -72,6 +71,7 @@ public HiveSchemaFactory(final HiveStoragePlugin plugin, final String name, fina
     isDrillImpersonationEnabled = plugin.getContext().getConfig().getBoolean(ExecConstants.IMPERSONATION_ENABLED);
 
     try {
+      // TODO: DRILL-6412. Clients for plugin should be instantiated only for the case, when plugin is enabled
       processUserMetastoreClient =
           DrillHiveMetaStoreClient.createCloseableClientWithCaching(hiveConf);
     } catch (MetaException e) {
@@ -82,12 +82,9 @@ public HiveSchemaFactory(final HiveStoragePlugin plugin, final String name, fina
         .newBuilder()
         .expireAfterAccess(10, TimeUnit.MINUTES)
         .maximumSize(5) // Up to 5 clients for impersonation-enabled.
-        .removalListener(new RemovalListener<String, DrillHiveMetaStoreClient>() {
-          @Override
-          public void onRemoval(RemovalNotification<String, DrillHiveMetaStoreClient> notification) {
-            DrillHiveMetaStoreClient client = notification.getValue();
-            client.close();
-          }
+        .removalListener((RemovalListener<String, DrillHiveMetaStoreClient>) notification -> {
+          DrillHiveMetaStoreClient client = notification.getValue();
+          client.close();
         })
         .build(new CacheLoader<String, DrillHiveMetaStoreClient>() {
           @Override
diff --git a/contrib/storage-hive/core/src/main/resources/bootstrap-storage-plugins.json b/contrib/storage-hive/core/src/main/resources/bootstrap-storage-plugins.json
index 5c7174e253..018189c382 100644
--- a/contrib/storage-hive/core/src/main/resources/bootstrap-storage-plugins.json
+++ b/contrib/storage-hive/core/src/main/resources/bootstrap-storage-plugins.json
@@ -2,14 +2,16 @@
   "storage":{
     hive : {
       type:"hive",
-      enabled: false,
-      config : {
+      configProps : {
         "hive.metastore.uris" : "",
         "javax.jdo.option.ConnectionURL" : "jdbc:derby:;databaseName=../sample-data/drill_hive_db;create=true",
         "hive.metastore.warehouse.dir" : "/tmp/drill_hive_wh",
         "fs.default.name" : "file:///",
-        "hive.metastore.sasl.enabled" : "false"
-      }
+        "hive.metastore.sasl.enabled" : "false",
+        "hive.metastore.schema.verification": "false",
+        "datanucleus.schema.autoCreateAll": "true"
+      },
+      enabled: false
     }
   }
 }
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/TestHiveDrillNativeParquetReader.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/TestHiveDrillNativeParquetReader.java
index fd9701cd16..ea8d5df841 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/TestHiveDrillNativeParquetReader.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/TestHiveDrillNativeParquetReader.java
@@ -42,13 +42,13 @@
 
   @BeforeClass
   public static void init() {
-    setSessionOption(ExecConstants.HIVE_OPTIMIZE_SCAN_WITH_NATIVE_READERS, true);
+    setSessionOption(ExecConstants.HIVE_OPTIMIZE_PARQUET_SCAN_WITH_NATIVE_READER, true);
     setSessionOption(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY, true);
   }
 
   @AfterClass
   public static void cleanup() {
-    resetSessionOption(ExecConstants.HIVE_OPTIMIZE_SCAN_WITH_NATIVE_READERS);
+    resetSessionOption(ExecConstants.HIVE_OPTIMIZE_PARQUET_SCAN_WITH_NATIVE_READER);
     resetSessionOption(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY);
   }
 
@@ -152,6 +152,12 @@ public void testPhysicalPlanSubmission() throws Exception {
     // checks only group scan
     PlanTestBase.testPhysicalPlanExecutionBasedOnQuery("select * from hive.kv_native");
     PlanTestBase.testPhysicalPlanExecutionBasedOnQuery("select * from hive.kv_native_ext");
+    try {
+      alterSession(ExecConstants.HIVE_CONF_PROPERTIES, "hive.mapred.supports.subdirectories=true\nmapred.input.dir.recursive=true");
+      PlanTestBase.testPhysicalPlanExecutionBasedOnQuery("select * from hive.sub_dir_table");
+    } finally {
+      resetSessionOption(ExecConstants.HIVE_CONF_PROPERTIES);
+    }
   }
 
   @Test
@@ -243,4 +249,15 @@ public void testNativeReaderIsDisabledForAlteredPartitionedTable() throws Except
     testPlanMatchingPatterns(query, new String[] {"HiveScan"}, new String[]{"HiveDrillNativeParquetScan"});
   }
 
+  @Test
+  public void testHiveConfPropertiesAtSessionLevel() throws Exception {
+    String query = "select * from hive.sub_dir_table";
+    try {
+      alterSession(ExecConstants.HIVE_CONF_PROPERTIES, "hive.mapred.supports.subdirectories=true\nmapred.input.dir.recursive=true");
+      test(query);
+    } finally {
+      resetSessionOption(ExecConstants.HIVE_CONF_PROPERTIES);
+    }
+  }
+
 }
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestHiveUDFs.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestHiveUDFs.java
index 305d9b574c..7843b3fe41 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestHiveUDFs.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestHiveUDFs.java
@@ -44,8 +44,6 @@
 
   @Test
   public void testGenericUDF() throws Throwable {
-
-    int numRecords = 0;
     String planString = Resources.toString(Resources.getResource("functions/hive/GenericUDF.json"), Charsets.UTF_8);
     List<QueryDataBatch> results = testPhysicalWithResults(planString);
 
@@ -82,10 +80,6 @@ public void testGenericUDF() throws Throwable {
         String concat = new String(concatV.getAccessor().get(i), Charsets.UTF_8);
         assertTrue(concat.equals(in+"-"+in));
 
-        float flt1 = flt1V.getAccessor().get(i);
-        String format_number = new String(format_numberV.getAccessor().get(i), Charsets.UTF_8);
-
-
         String nullableStr1 = null;
         if (!nullableStr1V.getAccessor().isNull(i)) {
           nullableStr1 = new String(nullableStr1V.getAccessor().get(i), Charsets.UTF_8);
@@ -100,23 +94,15 @@ public void testGenericUDF() throws Throwable {
         if (nullableStr1 != null) {
           assertEquals(nullableStr1.toUpperCase(), upperNullableStr1);
         }
-
-        System.out.println(in + ", " + upper + ", " + concat + ", " +
-          flt1 + ", " + format_number + ", " + nullableStr1 + ", " + upperNullableStr1);
-
-        numRecords++;
       }
 
       result.release();
       batchLoader.clear();
     }
-
-    System.out.println("Processed " + numRecords + " records");
   }
 
   @Test
   public void testUDF() throws Throwable {
-    int numRecords = 0;
     String planString = Resources.toString(Resources.getResource("functions/hive/UDF.json"), Charsets.UTF_8);
     List<QueryDataBatch> results = testPhysicalWithResults(planString);
 
@@ -146,8 +132,6 @@ public void testUDF() throws Throwable {
         long str1Length = str1LengthV.getAccessor().get(i);
         assertTrue(str1.length() == str1Length);
 
-        int str1Ascii = str1AsciiV.getAccessor().get(i);
-
         float flt1 = flt1V.getAccessor().get(i);
 
         double pow = 0;
@@ -155,16 +139,10 @@ public void testUDF() throws Throwable {
           pow = powV.getAccessor().get(i);
           assertTrue(Math.pow(flt1, 2.0) == pow);
         }
-
-        System.out.println(str1 + ", " + str1Length + ", " + str1Ascii + ", " + flt1 + ", " + pow);
-        numRecords++;
       }
 
       result.release();
       batchLoader.clear();
     }
-
-    System.out.println("Processed " + numRecords + " records");
   }
-
 }
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestHiveStorage.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestHiveStorage.java
index 25393e7363..94f39b806d 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestHiveStorage.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestHiveStorage.java
@@ -412,6 +412,27 @@ public void testNonAsciiStringLiterals() throws Exception {
   public void testPhysicalPlanSubmission() throws Exception {
     PlanTestBase.testPhysicalPlanExecutionBasedOnQuery("select * from hive.kv");
     PlanTestBase.testPhysicalPlanExecutionBasedOnQuery("select * from hive.readtest");
+    try {
+      alterSession(ExecConstants.HIVE_CONF_PROPERTIES, "hive.mapred.supports.subdirectories=true\nmapred.input.dir.recursive=true");
+      PlanTestBase.testPhysicalPlanExecutionBasedOnQuery("select * from hive.sub_dir_table");
+    } finally {
+      resetSessionOption(ExecConstants.HIVE_CONF_PROPERTIES);
+    }
+  }
+
+  @Test
+  public void testHiveConfPropertiesAtSessionLevel() throws Exception {
+    String query = "select * from hive.sub_dir_table";
+    try {
+      alterSession(ExecConstants.HIVE_CONF_PROPERTIES, "hive.mapred.supports.subdirectories=true\nmapred.input.dir.recursive=true");
+      test(query);
+    } finally {
+      resetSessionOption(ExecConstants.HIVE_CONF_PROPERTIES);
+    }
+
+    thrown.expect(UserRemoteException.class);
+    thrown.expectMessage(containsString("IOException: Not a file"));
+    test(query);
   }
 
   private void verifyColumnsMetadata(List<UserProtos.ResultColumnMetadata> columnsList, Map<String, Integer> expectedResult) {
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestInfoSchemaOnHiveStorage.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestInfoSchemaOnHiveStorage.java
index 80da976d44..c5c0d488c7 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestInfoSchemaOnHiveStorage.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestInfoSchemaOnHiveStorage.java
@@ -50,6 +50,7 @@ public void showTablesFromDb() throws Exception{
         .baselineValues("hive.default", "partition_with_few_schemas")
         .baselineValues("hive.default", "kv_native")
         .baselineValues("hive.default", "kv_native_ext")
+        .baselineValues("hive.default", "sub_dir_table")
         .go();
 
     testBuilder()
@@ -254,6 +255,7 @@ public void showInfoSchema() throws Exception {
         .baselineValues("DRILL", "hive.default", "simple_json", "TABLE")
         .baselineValues("DRILL", "hive.default", "kv_native", "TABLE")
         .baselineValues("DRILL", "hive.default", "kv_native_ext", "TABLE")
+        .baselineValues("DRILL", "hive.default", "sub_dir_table", "TABLE")
         .baselineValues("DRILL", "hive.skipper", "kv_text_small", "TABLE")
         .baselineValues("DRILL", "hive.skipper", "kv_text_large", "TABLE")
         .baselineValues("DRILL", "hive.skipper", "kv_incorrect_skip_header", "TABLE")
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/store/hive/HiveTestDataGenerator.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/store/hive/HiveTestDataGenerator.java
index f206999984..074cb3b83a 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/store/hive/HiveTestDataGenerator.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/store/hive/HiveTestDataGenerator.java
@@ -25,6 +25,7 @@
 import java.nio.file.attribute.PosixFilePermission;
 import java.sql.Date;
 import java.sql.Timestamp;
+import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
 
@@ -42,7 +43,6 @@
 import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.ql.session.SessionState;
 
-import com.google.common.collect.Maps;
 import org.apache.hadoop.hive.serde.serdeConstants;
 
 import static org.apache.drill.exec.hive.HiveTestUtilities.executeQuery;
@@ -80,7 +80,7 @@ private HiveTestDataGenerator(final String dbDir, final String whDir, final Base
     this.whDir = whDir;
     this.dirTestWatcher = dirTestWatcher;
 
-    config = Maps.newHashMap();
+    config = new HashMap<>();
     config.put(ConfVars.METASTOREURIS.toString(), "");
     config.put("javax.jdo.option.ConnectionURL", String.format("jdbc:derby:;databaseName=%s;create=true", dbDir));
     config.put("hive.metastore.warehouse.dir", whDir);
@@ -89,7 +89,9 @@ private HiveTestDataGenerator(final String dbDir, final String whDir, final Base
 
   /**
    * Add Hive test storage plugin to the given plugin registry.
-   * @throws Exception
+   *
+   * @param pluginRegistry storage plugin registry
+   * @throws Exception in case if unable to update Hive storage plugin
    */
   public void addHiveTestPlugin(final StoragePluginRegistry pluginRegistry) throws Exception {
     HiveStoragePluginConfig pluginConfig = new HiveStoragePluginConfig(config);
@@ -101,7 +103,8 @@ public void addHiveTestPlugin(final StoragePluginRegistry pluginRegistry) throws
   /**
    * Update the current HiveStoragePlugin in given plugin registry with given <i>configOverride</i>.
    *
-   * @param configOverride
+   * @param pluginRegistry storage plugin registry
+   * @param configOverride config properties to be overridden
    * @throws DrillException if fails to update or no Hive plugin currently exists in given plugin registry.
    */
   public void updatePluginConfig(final StoragePluginRegistry pluginRegistry, Map<String, String> configOverride)
@@ -113,7 +116,7 @@ public void updatePluginConfig(final StoragePluginRegistry pluginRegistry, Map<S
     }
 
     HiveStoragePluginConfig newPluginConfig = storagePlugin.getConfig();
-    newPluginConfig.getHiveConfigOverride().putAll(configOverride);
+    newPluginConfig.getConfigProps().putAll(configOverride);
 
     pluginRegistry.createOrUpdate(HIVE_TEST_PLUGIN_NAME, newPluginConfig, true);
   }
@@ -344,7 +347,7 @@ private void generateTestData() throws Exception {
         "charType CHAR(10))"
     );
 
-    /**
+    /*
      * Create a PARQUET table with all supported types.
      */
     executeQuery(hiveDriver,
@@ -542,6 +545,8 @@ private void generateTestData() throws Exception {
 
     createTestDataForDrillNativeParquetReaderTests(hiveDriver);
 
+    createSubDirTable(hiveDriver, testDataFile);
+
     ss.close();
   }
 
@@ -594,56 +599,61 @@ private void createTestDataForDrillNativeParquetReaderTests(Driver hiveDriver) {
       "location '%s'", thirdPartition));
   }
 
+  private void createSubDirTable(Driver hiveDriver, String testDataFile) {
+    String tableName = "sub_dir_table";
+    dirTestWatcher.copyResourceToRoot(Paths.get(testDataFile), Paths.get(tableName, "sub_dir", "data.txt"));
+
+    String tableLocation = Paths.get(dirTestWatcher.getRootDir().toURI().getPath(), tableName).toUri().getPath();
+
+    String tableDDL = String.format("create external table sub_dir_table (key int, value string) " +
+        "row format delimited fields terminated by ',' stored as textfile location '%s'", tableLocation);
+    executeQuery(hiveDriver, tableDDL);
+  }
+
   private File getTempFile() throws Exception {
     return java.nio.file.Files.createTempFile("drill-hive-test", ".txt").toFile();
   }
 
   private String generateTestDataFile() throws Exception {
-    final File file = getTempFile();
-    PrintWriter printWriter = new PrintWriter(file);
-    for (int i=1; i<=5; i++) {
-      printWriter.println (String.format("%d, key_%d", i, i));
+    File file = getTempFile();
+    try (PrintWriter printWriter = new PrintWriter(file)) {
+      for (int i = 1; i <= 5; i++) {
+        printWriter.println(String.format("%d, key_%d", i, i));
+      }
     }
-    printWriter.close();
-
     return file.getPath();
   }
 
   private String generateTestDataFileForPartitionInput() throws Exception {
-    final File file = getTempFile();
-
-    PrintWriter printWriter = new PrintWriter(file);
-
-    String partValues[] = {"1", "2", "null"};
-
-    for(int c = 0; c < partValues.length; c++) {
-      for(int d = 0; d < partValues.length; d++) {
-        for(int e = 0; e < partValues.length; e++) {
-          for (int i = 1; i <= 5; i++) {
-            Date date = new Date(System.currentTimeMillis());
-            Timestamp ts = new Timestamp(System.currentTimeMillis());
-            printWriter.printf("%s,%s,%s,%s,%s",
-                date.toString(), ts.toString(), partValues[c], partValues[d], partValues[e]);
-            printWriter.println();
+    File file = getTempFile();
+    try (PrintWriter printWriter = new PrintWriter(file)) {
+      String partValues[] = {"1", "2", "null"};
+      for (String partValue : partValues) {
+        for (String partValue1 : partValues) {
+          for (String partValue2 : partValues) {
+            for (int i = 1; i <= 5; i++) {
+              Date date = new Date(System.currentTimeMillis());
+              Timestamp ts = new Timestamp(System.currentTimeMillis());
+              printWriter.printf("%s,%s,%s,%s,%s", date.toString(), ts.toString(), partValue, partValue1, partValue2);
+              printWriter.println();
+            }
           }
         }
       }
     }
 
-    printWriter.close();
-
     return file.getPath();
   }
 
   private String generateAllTypesDataFile() throws Exception {
     File file = getTempFile();
 
-    PrintWriter printWriter = new PrintWriter(file);
-    printWriter.println("YmluYXJ5ZmllbGQ=,false,34,65.99,2347.923,2758725827.9999,29375892739852.7689," +
-        "89853749534593985.7834783,8.345,4.67,123456,234235,3455,stringfield,varcharfield," +
-        "2013-07-05 17:01:00,2013-07-05,charfield");
-    printWriter.println(",,,,,,,,,,,,,,,,");
-    printWriter.close();
+    try (PrintWriter printWriter = new PrintWriter(file)) {
+      printWriter.println("YmluYXJ5ZmllbGQ=,false,34,65.99,2347.923,2758725827.9999,29375892739852.7689,"+
+          "89853749534593985.7834783,8.345,4.67,123456,234235,3455,stringfield,varcharfield,"+
+          "2013-07-05 17:01:00,2013-07-05,charfield");
+      printWriter.println(",,,,,,,,,,,,,,,,");
+    }
 
     return file.getPath();
   }
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/test/Drill2130StorageHiveCoreHamcrestConfigurationTest.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/test/Drill2130StorageHiveCoreHamcrestConfigurationTest.java
index 472891868c..12adf92e5f 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/test/Drill2130StorageHiveCoreHamcrestConfigurationTest.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/test/Drill2130StorageHiveCoreHamcrestConfigurationTest.java
@@ -24,6 +24,7 @@
 import static org.hamcrest.CoreMatchers.equalTo;
 
 public class Drill2130StorageHiveCoreHamcrestConfigurationTest {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Drill2130StorageHiveCoreHamcrestConfigurationTest.class);
 
   @SuppressWarnings("unused")
   private org.hamcrest.MatcherAssert forCompileTimeCheckForNewEnoughHamcrest;
@@ -38,9 +39,7 @@ public void testJUnitHamcrestMatcherFailureWorks() {
              + "  Got NoSuchMethodError;  e: " + e );
     }
     catch ( AssertionError e ) {
-      System.out.println( "Class path seems fine re new JUnit vs. old Hamcrest."
-                          + " (Got AssertionError, not NoSuchMethodError.)" );
+      logger.info("Class path seems fine re new JUnit vs. old Hamcrest. (Got AssertionError, not NoSuchMethodError.)");
     }
   }
-
 }
diff --git a/contrib/storage-hive/hive-exec-shade/pom.xml b/contrib/storage-hive/hive-exec-shade/pom.xml
index 6f511adf71..98fd4b8150 100644
--- a/contrib/storage-hive/hive-exec-shade/pom.xml
+++ b/contrib/storage-hive/hive-exec-shade/pom.xml
@@ -31,6 +31,20 @@
   <packaging>jar</packaging>
   <name>contrib/hive-storage-plugin/hive-exec-shaded</name>
 
+  <properties>
+    <hive.parquet.version>1.8.3</hive.parquet.version>
+  </properties>
+
+  <dependencyManagement>
+    <dependencies>
+      <dependency>
+        <groupId>org.apache.parquet</groupId>
+        <artifactId>parquet-hadoop-bundle</artifactId>
+        <version>${hive.parquet.version}</version>
+      </dependency>
+    </dependencies>
+  </dependencyManagement>
+
   <dependencies>
     <dependency>
       <groupId>org.apache.hive</groupId>
@@ -68,11 +82,6 @@
         </exclusion>
       </exclusions>
     </dependency>
-    <!--Once newer hive-exec version leverages parquet-column 1.9.0, this dependency can be deleted -->
-    <dependency>
-      <groupId>org.apache.parquet</groupId>
-      <artifactId>parquet-column</artifactId>
-    </dependency>
   </dependencies>
 
   <build>
@@ -83,7 +92,7 @@
           <artifactSet>
             <includes>
               <include>org.apache.hive:hive-exec</include>
-              <include>org.apache.parquet:parquet-column</include>
+              <include>org.apache.parquet:parquet-hadoop-bundle</include>
               <include>commons-codec:commons-codec</include>
               <include>com.fasterxml.jackson.core:jackson-databind</include>
               <include>com.fasterxml.jackson.core:jackson-annotations</include>
@@ -117,6 +126,10 @@
               <pattern>org.apache.parquet.</pattern>
               <shadedPattern>hive.org.apache.parquet.</shadedPattern>
             </relocation>
+            <relocation>
+              <pattern>shaded.parquet.</pattern>
+              <shadedPattern>hive.shaded.parquet.</shadedPattern>
+            </relocation>
             <relocation>
               <pattern>org.apache.avro.</pattern>
               <shadedPattern>hive.org.apache.avro.</shadedPattern>
diff --git a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcPrel.java b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcPrel.java
index abeca2302d..ac6f31c4e7 100644
--- a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcPrel.java
+++ b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcPrel.java
@@ -44,7 +44,6 @@
  * Represents a JDBC Plan once the children nodes have been rewritten into SQL.
  */
 public class JdbcPrel extends AbstractRelNode implements Prel {
-
   private final String sql;
   private final double rows;
   private final DrillJdbcConvention convention;
@@ -66,6 +65,18 @@ public JdbcPrel(RelOptCluster cluster, RelTraitSet traitSet, JdbcIntermediatePre
     rowType = input.getRowType();
   }
 
+  //Substitute newline. Also stripping away single line comments. Expecting hints to be nested in '/* <hint> */'
+  private String stripToOneLineSql(String sql) {
+    StringBuilder strippedSqlTextBldr = new StringBuilder(sql.length());
+    String sqlToken[] = sql.split("\\n");
+    for (String sqlTextLine : sqlToken) {
+      if (!sqlTextLine.trim().startsWith("--")) { //Skip comments
+        strippedSqlTextBldr.append(sqlTextLine).append(' ');
+      }
+    }
+    return strippedSqlTextBldr.toString();
+  }
+
   private class SubsetRemover extends RelShuttleImpl {
 
     @Override
@@ -87,7 +98,7 @@ public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws
 
   @Override
   public RelWriter explainTerms(RelWriter pw) {
-    return super.explainTerms(pw).item("sql", sql);
+    return super.explainTerms(pw).item("sql", stripToOneLineSql(sql));
   }
 
   @Override
diff --git a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcSubScan.java b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcSubScan.java
index 65fa4fdabb..34d18273e6 100755
--- a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcSubScan.java
+++ b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcSubScan.java
@@ -20,6 +20,7 @@
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.exec.physical.base.AbstractSubScan;
+import org.apache.drill.exec.proto.beans.CoreOperatorType;
 import org.apache.drill.exec.store.StoragePluginRegistry;
 
 import com.fasterxml.jackson.annotation.JacksonInject;
@@ -52,7 +53,7 @@ public JdbcSubScan(
 
   @Override
   public int getOperatorType() {
-    return -1;
+    return CoreOperatorType.JDBC_SCAN.getNumber();
   }
 
   public String getSql() {
diff --git a/contrib/storage-jdbc/src/test/resources/bootstrap-storage-plugins.json b/contrib/storage-jdbc/src/test/resources/bootstrap-storage-plugins.json
index add9808474..4018d92478 100755
--- a/contrib/storage-jdbc/src/test/resources/bootstrap-storage-plugins.json
+++ b/contrib/storage-jdbc/src/test/resources/bootstrap-storage-plugins.json
@@ -2,15 +2,16 @@
     "storage" : {
         derby : {
           type    : "jdbc",
-          enabled : true,
           driver  : "org.apache.derby.jdbc.ClientDriver",
-          url     : "jdbc:derby://localhost:${derby.reserved.port}/memory:${derby.database.name};user=root;password=root"
+          url     : "jdbc:derby://localhost:${derby.reserved.port}/memory:${derby.database.name};user=root;password=root",
+          enabled : true
         },
         mysql : {
           type    : "jdbc",
           enabled : true,
           driver  : "com.mysql.jdbc.Driver",
-          url     : "jdbc:mysql://localhost:${mysql.reserved.port}/${mysql.database.name}?user=root&password=root&useJDBCCompliantTimezoneShift=true"
+          url     : "jdbc:mysql://localhost:${mysql.reserved.port}/${mysql.database.name}?user=root&password=root&useJDBCCompliantTimezoneShift=true",
+          enabled : true
       }
     }
 }
diff --git a/contrib/storage-kafka/pom.xml b/contrib/storage-kafka/pom.xml
index a7f6ed0323..5e2809f2eb 100644
--- a/contrib/storage-kafka/pom.xml
+++ b/contrib/storage-kafka/pom.xml
@@ -88,14 +88,6 @@
     <dependency>
       <groupId>org.apache.curator</groupId>
       <artifactId>curator-test</artifactId>
-      <version>3.3.0</version>
-      <scope>test</scope>
-      <exclusions>
-        <exclusion>
-          <groupId>log4j</groupId>
-          <artifactId>log4j</artifactId>
-        </exclusion>
-      </exclusions>
     </dependency>
     <dependency>
       <groupId>org.apache.drill</groupId>
diff --git a/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaGroupScan.java b/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaGroupScan.java
index 9cf575b345..976c82a0ff 100644
--- a/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaGroupScan.java
+++ b/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaGroupScan.java
@@ -19,10 +19,12 @@
 
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import com.google.common.collect.Sets;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.exceptions.UserException;
@@ -35,7 +37,6 @@
 import org.apache.drill.exec.physical.base.ScanStats.GroupScanProperty;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.store.StoragePluginRegistry;
-import org.apache.drill.exec.store.kafka.KafkaSubScan.KafkaSubScanSpec;
 import org.apache.drill.exec.store.schedule.AffinityCreator;
 import org.apache.drill.exec.store.schedule.AssignmentCreator;
 import org.apache.drill.exec.store.schedule.CompleteWork;
@@ -72,10 +73,11 @@
   private final KafkaScanSpec kafkaScanSpec;
 
   private List<SchemaPath> columns;
-  private List<PartitionScanWork> partitionWorkList;
   private ListMultimap<Integer, PartitionScanWork> assignments;
   private List<EndpointAffinity> affinities;
 
+  private Map<TopicPartition, PartitionScanWork> partitionWorkMap;
+
   @JsonCreator
   public KafkaGroupScan(@JsonProperty("userName") String userName,
                         @JsonProperty("kafkaStoragePluginConfig") KafkaStoragePluginConfig kafkaStoragePluginConfig,
@@ -112,34 +114,18 @@ public KafkaGroupScan(KafkaGroupScan that) {
     this.kafkaStoragePlugin = that.kafkaStoragePlugin;
     this.columns = that.columns;
     this.kafkaScanSpec = that.kafkaScanSpec;
-    this.partitionWorkList = that.partitionWorkList;
     this.assignments = that.assignments;
+    this.partitionWorkMap = that.partitionWorkMap;
   }
 
-  private static class PartitionScanWork implements CompleteWork {
-
-    private final EndpointByteMapImpl byteMap = new EndpointByteMapImpl();
-
-    private final TopicPartition topicPartition;
-    private final long beginOffset;
-    private final long latestOffset;
-
-    public PartitionScanWork(TopicPartition topicPartition, long beginOffset, long latestOffset) {
-      this.topicPartition = topicPartition;
-      this.beginOffset = beginOffset;
-      this.latestOffset = latestOffset;
-    }
-
-    public TopicPartition getTopicPartition() {
-      return topicPartition;
-    }
+  public static class PartitionScanWork implements CompleteWork {
 
-    public long getBeginOffset() {
-      return beginOffset;
-    }
+    private final EndpointByteMapImpl byteMap;
+    private final KafkaPartitionScanSpec partitionScanSpec;
 
-    public long getLatestOffset() {
-      return latestOffset;
+    public PartitionScanWork(EndpointByteMap byteMap, KafkaPartitionScanSpec partitionScanSpec) {
+      this.byteMap = (EndpointByteMapImpl)byteMap;
+      this.partitionScanSpec = partitionScanSpec;
     }
 
     @Override
@@ -149,7 +135,7 @@ public int compareTo(CompleteWork o) {
 
     @Override
     public long getTotalBytes() {
-      return (latestOffset - beginOffset) * MSG_SIZE;
+      return (partitionScanSpec.getEndOffset() - partitionScanSpec.getStartOffset()) * MSG_SIZE;
     }
 
     @Override
@@ -157,6 +143,9 @@ public EndpointByteMap getByteMap() {
       return byteMap;
     }
 
+    public KafkaPartitionScanSpec getPartitionScanSpec() {
+      return partitionScanSpec;
+    }
   }
 
   /**
@@ -164,7 +153,7 @@ public EndpointByteMap getByteMap() {
    * corresponding topicPartition
    */
   private void init() {
-    partitionWorkList = Lists.newArrayList();
+    partitionWorkMap = Maps.newHashMap();
     Collection<DrillbitEndpoint> endpoints = kafkaStoragePlugin.getContext().getBits();
     Map<String, DrillbitEndpoint> endpointMap = Maps.newHashMap();
     for (DrillbitEndpoint endpoint : endpoints) {
@@ -211,12 +200,13 @@ private void init() {
 
     // computes work for each end point
     for (PartitionInfo partitionInfo : topicPartitions) {
-      TopicPartition topicPartition = new TopicPartition(topicName, partitionInfo.partition());
+      TopicPartition topicPartition = new TopicPartition(partitionInfo.topic(), partitionInfo.partition());
       long lastCommittedOffset = startOffsetsMap.get(topicPartition);
       long latestOffset = endOffsetsMap.get(topicPartition);
       logger.debug("Latest offset of {} is {}", topicPartition, latestOffset);
       logger.debug("Last committed offset of {} is {}", topicPartition, lastCommittedOffset);
-      PartitionScanWork work = new PartitionScanWork(topicPartition, lastCommittedOffset, latestOffset);
+      KafkaPartitionScanSpec partitionScanSpec = new KafkaPartitionScanSpec(topicPartition.topic(), topicPartition.partition(), lastCommittedOffset, latestOffset);
+      PartitionScanWork work = new PartitionScanWork(new EndpointByteMapImpl(), partitionScanSpec);
       Node[] inSyncReplicas = partitionInfo.inSyncReplicas();
       for (Node isr : inSyncReplicas) {
         String host = isr.host();
@@ -225,23 +215,22 @@ private void init() {
           work.getByteMap().add(ep, work.getTotalBytes());
         }
       }
-      partitionWorkList.add(work);
+      partitionWorkMap.put(topicPartition, work);
     }
   }
 
   @Override
   public void applyAssignments(List<DrillbitEndpoint> incomingEndpoints) {
-    assignments = AssignmentCreator.getMappings(incomingEndpoints, partitionWorkList);
+    assignments = AssignmentCreator.getMappings(incomingEndpoints, Lists.newArrayList(partitionWorkMap.values()));
   }
 
   @Override
   public KafkaSubScan getSpecificScan(int minorFragmentId) {
     List<PartitionScanWork> workList = assignments.get(minorFragmentId);
-    List<KafkaSubScanSpec> scanSpecList = Lists.newArrayList();
+    List<KafkaPartitionScanSpec> scanSpecList = Lists.newArrayList();
 
     for (PartitionScanWork work : workList) {
-      scanSpecList.add(new KafkaSubScanSpec(work.getTopicPartition().topic(), work.getTopicPartition().partition(),
-          work.getBeginOffset(), work.getLatestOffset()));
+      scanSpecList.add(work.partitionScanSpec);
     }
 
     return new KafkaSubScan(getUserName(), kafkaStoragePlugin, columns, scanSpecList);
@@ -249,14 +238,14 @@ public KafkaSubScan getSpecificScan(int minorFragmentId) {
 
   @Override
   public int getMaxParallelizationWidth() {
-    return partitionWorkList.size();
+    return partitionWorkMap.values().size();
   }
 
   @Override
   public ScanStats getScanStats() {
     long messageCount = 0;
-    for (PartitionScanWork work : partitionWorkList) {
-      messageCount += (work.getLatestOffset() - work.getBeginOffset());
+    for (PartitionScanWork work : partitionWorkMap.values()) {
+      messageCount += (work.getPartitionScanSpec().getEndOffset() - work.getPartitionScanSpec().getStartOffset());
     }
     return new ScanStats(GroupScanProperty.EXACT_ROW_COUNT, messageCount, 1, messageCount * MSG_SIZE);
   }
@@ -275,7 +264,7 @@ public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) thro
   @Override
   public List<EndpointAffinity> getOperatorAffinity() {
     if (affinities == null) {
-      affinities = AffinityCreator.getAffinityMap(partitionWorkList);
+      affinities = AffinityCreator.getAffinityMap(Lists.newArrayList(partitionWorkMap.values()));
     }
     return affinities;
   }
@@ -293,6 +282,23 @@ public GroupScan clone(List<SchemaPath> columns) {
     return clone;
   }
 
+  public GroupScan cloneWithNewSpec(List<KafkaPartitionScanSpec> partitionScanSpecList) {
+    KafkaGroupScan clone = new KafkaGroupScan(this);
+    HashSet<TopicPartition> partitionsInSpec = Sets.newHashSet();
+
+    for(KafkaPartitionScanSpec scanSpec : partitionScanSpecList) {
+      TopicPartition tp = new TopicPartition(scanSpec.getTopicName(), scanSpec.getPartitionId());
+      partitionsInSpec.add(tp);
+
+      PartitionScanWork newScanWork = new PartitionScanWork(partitionWorkMap.get(tp).getByteMap(), scanSpec);
+      clone.partitionWorkMap.put(tp, newScanWork);
+    }
+
+    //Remove unnecessary partitions from partitionWorkMap
+    clone.partitionWorkMap.keySet().removeIf(tp -> !partitionsInSpec.contains(tp));
+    return clone;
+  }
+
   @JsonProperty
   public KafkaStoragePluginConfig getKafkaStoragePluginConfig() {
     return kafkaStoragePlugin.getConfig();
@@ -318,4 +324,12 @@ public String toString() {
     return String.format("KafkaGroupScan [KafkaScanSpec=%s, columns=%s]", kafkaScanSpec, columns);
   }
 
+  @JsonIgnore
+  public List<KafkaPartitionScanSpec> getPartitionScanSpecList() {
+    List<KafkaPartitionScanSpec> partitionScanSpecList = Lists.newArrayList();
+    for (PartitionScanWork work : partitionWorkMap.values()) {
+      partitionScanSpecList.add(work.partitionScanSpec.clone());
+    }
+    return partitionScanSpecList;
+  }
 }
diff --git a/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaNodeProcessor.java b/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaNodeProcessor.java
new file mode 100644
index 0000000000..ba39b76b8f
--- /dev/null
+++ b/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaNodeProcessor.java
@@ -0,0 +1,186 @@
+/*
+ * 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.kafka;
+
+import org.apache.drill.common.expression.FunctionCall;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.expression.ValueExpressions.BooleanExpression;
+import org.apache.drill.common.expression.ValueExpressions.DateExpression;
+import org.apache.drill.common.expression.ValueExpressions.DoubleExpression;
+import org.apache.drill.common.expression.ValueExpressions.FloatExpression;
+import org.apache.drill.common.expression.ValueExpressions.IntExpression;
+import org.apache.drill.common.expression.ValueExpressions.LongExpression;
+import org.apache.drill.common.expression.ValueExpressions.QuotedString;
+import org.apache.drill.common.expression.ValueExpressions.TimeExpression;
+import org.apache.drill.common.expression.ValueExpressions.TimeStampExpression;
+import org.apache.drill.common.expression.visitors.AbstractExprVisitor;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+
+class KafkaNodeProcessor extends AbstractExprVisitor<Boolean, LogicalExpression, RuntimeException> {
+
+  private String functionName;
+  private Boolean success;
+  private Long value;
+  private String path;
+
+  public KafkaNodeProcessor(String functionName) {
+    this.functionName = functionName;
+    this.success = false;
+  }
+
+  public static boolean isPushdownFunction(String functionName) {
+    return COMPARE_FUNCTIONS_TRANSPOSE_MAP.keySet().contains(functionName);
+  }
+
+  @Override
+  public Boolean visitUnknown(LogicalExpression e, LogicalExpression valueArg) throws RuntimeException {
+    return false;
+  }
+
+  public static KafkaNodeProcessor process(FunctionCall call) {
+    String functionName = call.getName();
+    LogicalExpression nameArg = call.args.get(0);
+    LogicalExpression valueArg = call.args.size() >= 2? call.args.get(1) : null;
+    KafkaNodeProcessor evaluator = new KafkaNodeProcessor(functionName);
+
+    if (VALUE_EXPRESSION_CLASSES.contains(nameArg.getClass())) {
+      LogicalExpression swapArg = valueArg;
+      valueArg = nameArg;
+      nameArg = swapArg;
+      evaluator.functionName = COMPARE_FUNCTIONS_TRANSPOSE_MAP.get(functionName);
+    }
+    evaluator.success = nameArg.accept(evaluator, valueArg);
+    return evaluator;
+  }
+
+  public boolean isSuccess() {
+    // TODO Auto-generated method stub
+    return success;
+  }
+
+  public String getPath() {
+    return path;
+  }
+
+  public Long getValue() {
+    return value;
+  }
+
+  public String getFunctionName() {
+    return functionName;
+  }
+
+  @Override
+  public Boolean visitSchemaPath(SchemaPath path, LogicalExpression valueArg) throws RuntimeException {
+    this.path = path.getRootSegmentPath();
+
+    if(valueArg == null) {
+      return false;
+    }
+
+    switch (this.path) {
+      case "kafkaMsgOffset":
+        /*
+         * Do not pushdown "not_equal" on kafkaMsgOffset.
+         */
+        if(functionName.equals("not_equal")) {
+          return false;
+        }
+      case "kafkaPartitionId":
+        if(valueArg instanceof IntExpression) {
+          value = (long) ((IntExpression) valueArg).getInt();
+          return true;
+        }
+
+        if(valueArg instanceof LongExpression) {
+          value = ((LongExpression) valueArg).getLong();
+          return true;
+        }
+        break;
+      case "kafkaMsgTimestamp":
+        /*
+        Only pushdown "equal", "greater_than", "greater_than_or_equal" on kafkaMsgTimestamp
+         */
+        if(!functionName.equals("equal") && !functionName.equals("greater_than")
+               && !functionName.equals("greater_than_or_equal_to")) {
+          return false;
+        }
+
+        if(valueArg instanceof LongExpression) {
+          value = ((LongExpression) valueArg).getLong();
+          return true;
+        }
+
+        if (valueArg instanceof DateExpression) {
+          value = ((DateExpression)valueArg).getDate();
+          return true;
+        }
+
+        if (valueArg instanceof TimeExpression) {
+          value = (long) ((TimeExpression)valueArg).getTime();
+          return true;
+        }
+
+        if (valueArg instanceof TimeStampExpression) {
+          value = ((TimeStampExpression) valueArg).getTimeStamp();
+          return true;
+        }
+
+        if(valueArg instanceof IntExpression) {
+          value = (long) ((IntExpression) valueArg).getInt();
+          return true;
+        }
+        break;
+    }
+    return false;
+  }
+
+  private static final ImmutableSet<Class<? extends LogicalExpression>> VALUE_EXPRESSION_CLASSES;
+  static {
+    ImmutableSet.Builder<Class<? extends LogicalExpression>> builder = ImmutableSet.builder();
+    VALUE_EXPRESSION_CLASSES = builder
+                                   .add(BooleanExpression.class)
+                                   .add(DateExpression.class)
+                                   .add(DoubleExpression.class)
+                                   .add(FloatExpression.class)
+                                   .add(IntExpression.class)
+                                   .add(LongExpression.class)
+                                   .add(QuotedString.class)
+                                   .add(TimeExpression.class)
+                                   .build();
+  }
+
+  private static final ImmutableMap<String, String> COMPARE_FUNCTIONS_TRANSPOSE_MAP;
+  static {
+    ImmutableMap.Builder<String, String> builder = ImmutableMap.builder();
+    COMPARE_FUNCTIONS_TRANSPOSE_MAP = builder
+                                          .put("equal", "equal")
+                                          .put("not_equal", "not_equal")
+                                          .put("greater_than_or_equal_to", "less_than_or_equal_to")
+                                          .put("greater_than", "less_than")
+                                          .put("less_than_or_equal_to", "greater_than_or_equal_to")
+                                          .put("less_than", "greater_than")
+                                          .build();
+  }
+
+}
+
+
diff --git a/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaPartitionScanSpec.java b/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaPartitionScanSpec.java
new file mode 100644
index 0000000000..713f62e9bb
--- /dev/null
+++ b/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaPartitionScanSpec.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.store.kafka;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+public class KafkaPartitionScanSpec {
+  private String topicName;
+  private int partitionId;
+  private long startOffset;
+  private long endOffset;
+
+  @JsonCreator
+  public KafkaPartitionScanSpec(@JsonProperty("topicName") String topicName,
+                                @JsonProperty("partitionId") int partitionId,
+                                @JsonProperty("startOffset") long startOffset,
+                                @JsonProperty("endOffset") long endOffset) {
+    this.topicName = topicName;
+    this.partitionId = partitionId;
+    this.startOffset = startOffset;
+    this.endOffset = endOffset;
+  }
+
+  public String getTopicName() {
+    return topicName;
+  }
+
+  public int getPartitionId() {
+    return partitionId;
+  }
+
+  public long getStartOffset() {
+    return startOffset;
+  }
+
+  public long getEndOffset() {
+    return endOffset;
+  }
+
+  public void mergeScanSpec(String functionName, KafkaPartitionScanSpec scanSpec) {
+    switch (functionName) {
+      case "booleanAnd":
+        //Reduce the scan range
+        if(startOffset < scanSpec.startOffset) {
+          startOffset = scanSpec.startOffset;
+        }
+
+        if(endOffset > scanSpec.endOffset) {
+          endOffset = scanSpec.endOffset;
+        }
+        break;
+      case "booleanOr":
+        //Increase the scan range
+        if(scanSpec.startOffset < startOffset) {
+          startOffset = scanSpec.startOffset;
+        }
+
+        if(scanSpec.endOffset > endOffset) {
+          endOffset = scanSpec.endOffset;
+        }
+        break;
+    }
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if(obj instanceof KafkaPartitionScanSpec) {
+      KafkaPartitionScanSpec that = ((KafkaPartitionScanSpec)obj);
+      return this.topicName.equals(that.topicName) && this.partitionId == that.partitionId
+                 && this.startOffset == that.startOffset && this.endOffset == that.endOffset;
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return "KafkaPartitionScanSpec [topicName=" + topicName + ", partitionId=" + partitionId + ", startOffset="
+               + startOffset + ", endOffset=" + endOffset + "]";
+  }
+
+  public KafkaPartitionScanSpec clone() {
+    return new KafkaPartitionScanSpec(topicName, partitionId, startOffset, endOffset);
+  }
+}
diff --git a/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaPartitionScanSpecBuilder.java b/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaPartitionScanSpecBuilder.java
new file mode 100644
index 0000000000..b52ed44208
--- /dev/null
+++ b/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaPartitionScanSpecBuilder.java
@@ -0,0 +1,345 @@
+/*
+ * 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.kafka;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import org.apache.drill.common.expression.BooleanOperator;
+import org.apache.drill.common.expression.FunctionCall;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.visitors.AbstractExprVisitor;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.consumer.OffsetAndTimestamp;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+public class KafkaPartitionScanSpecBuilder extends
+    AbstractExprVisitor<List<KafkaPartitionScanSpec>,Void,RuntimeException> {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(KafkaPartitionScanSpecBuilder.class);
+  private final LogicalExpression le;
+  private final KafkaGroupScan groupScan;
+  private final KafkaConsumer<? ,?> kafkaConsumer;
+  private ImmutableMap<TopicPartition, KafkaPartitionScanSpec> fullScanSpec;
+  private static final long CLOSE_TIMEOUT_MS = 200;
+
+  public KafkaPartitionScanSpecBuilder(KafkaGroupScan groupScan, LogicalExpression conditionExp) {
+    this.groupScan = groupScan;
+    kafkaConsumer = new KafkaConsumer<>(groupScan.getKafkaStoragePluginConfig().getKafkaConsumerProps(),
+        new ByteArrayDeserializer(), new ByteArrayDeserializer());
+    le = conditionExp;
+  }
+
+  public List<KafkaPartitionScanSpec> parseTree() {
+    ImmutableMap.Builder<TopicPartition, KafkaPartitionScanSpec> builder = ImmutableMap.builder();
+    for(KafkaPartitionScanSpec scanSpec : groupScan.getPartitionScanSpecList()) {
+      builder.put(new TopicPartition(scanSpec.getTopicName(), scanSpec.getPartitionId()), scanSpec);
+    }
+    fullScanSpec = builder.build();
+    List<KafkaPartitionScanSpec> pushdownSpec = le.accept(this, null);
+
+    /*
+    Non-existing / invalid partitions may result in empty scan spec.
+    This results in a "ScanBatch" with no reader. DRILL currently requires
+    at least one reader to be present in a scan batch.
+     */
+    if(pushdownSpec != null && pushdownSpec.isEmpty()) {
+      TopicPartition firstPartition = new TopicPartition(groupScan.getKafkaScanSpec().getTopicName(), 0);
+      KafkaPartitionScanSpec emptySpec =
+          new KafkaPartitionScanSpec(firstPartition.topic(),firstPartition.partition(),
+              fullScanSpec.get(firstPartition).getEndOffset(), fullScanSpec.get(firstPartition).getEndOffset());
+      pushdownSpec.add(emptySpec);
+    }
+    return pushdownSpec;
+  }
+
+  @Override
+  public List<KafkaPartitionScanSpec> visitUnknown(LogicalExpression e, Void value)
+      throws RuntimeException {
+    return null;
+  }
+
+  @Override
+  public List<KafkaPartitionScanSpec> visitBooleanOperator(BooleanOperator op, Void value)
+      throws RuntimeException {
+
+    Map<TopicPartition, KafkaPartitionScanSpec> specMap = Maps.newHashMap();
+    ImmutableList<LogicalExpression> args = op.args;
+    if(op.getName().equals("booleanOr")) {
+
+      for(LogicalExpression expr : args) {
+        List<KafkaPartitionScanSpec> parsedSpec = expr.accept(this, null);
+        //parsedSpec is null if expression cannot be pushed down
+        if(parsedSpec != null) {
+          for(KafkaPartitionScanSpec newSpec : parsedSpec) {
+            TopicPartition tp = new TopicPartition(newSpec.getTopicName(), newSpec.getPartitionId());
+            KafkaPartitionScanSpec existingSpec = specMap.get(tp);
+
+            //If existing spec does not contain topic-partition
+            if(existingSpec == null) {
+              specMap.put(tp, newSpec); //Add topic-partition to spec for OR
+            } else {
+              existingSpec.mergeScanSpec(op.getName(), newSpec);
+              specMap.put(tp, existingSpec);
+            }
+          }
+        } else {
+          return null; //At any level, all arguments of booleanOr should support pushdown, else return null
+        }
+      }
+    } else { //booleanAnd
+      specMap.putAll(fullScanSpec);
+      for(LogicalExpression expr : args) {
+        List<KafkaPartitionScanSpec> parsedSpec = expr.accept(this, null);
+
+        //parsedSpec is null if expression cannot be pushed down
+        if(parsedSpec != null) {
+          Set<TopicPartition> partitionsInNewSpec = Sets.newHashSet(); //Store topic-partitions returned from new spec.
+
+          for (KafkaPartitionScanSpec newSpec : parsedSpec) {
+            TopicPartition tp = new TopicPartition(newSpec.getTopicName(), newSpec.getPartitionId());
+            partitionsInNewSpec.add(tp);
+            KafkaPartitionScanSpec existingSpec = specMap.get(tp);
+
+            if (existingSpec != null) {
+              existingSpec.mergeScanSpec(op.getName(), newSpec);
+              specMap.put(tp, existingSpec);
+            }
+          }
+
+          /*
+          For "booleanAnd", handle the case where condition is on `kafkaPartitionId`.
+          In this case, we would not want unnecessarily scan all the topic-partitions.
+          Hence we remove the unnecessary topic-partitions from the spec.
+         */
+          specMap.keySet().removeIf(partition -> !partitionsInNewSpec.contains(partition));
+        }
+
+      }
+    }
+    return Lists.newArrayList(specMap.values());
+  }
+
+  @Override
+  public List<KafkaPartitionScanSpec> visitFunctionCall(FunctionCall call, Void value)
+      throws RuntimeException {
+
+    String functionName = call.getName();
+    if(KafkaNodeProcessor.isPushdownFunction(functionName)) {
+
+      KafkaNodeProcessor kafkaNodeProcessor = KafkaNodeProcessor.process(call);
+      if(kafkaNodeProcessor.isSuccess()) {
+        switch (kafkaNodeProcessor.getPath()) {
+          case "kafkaMsgTimestamp":
+            return createScanSpecForTimestamp(kafkaNodeProcessor.getFunctionName(),
+                kafkaNodeProcessor.getValue());
+          case "kafkaMsgOffset":
+            return createScanSpecForOffset(kafkaNodeProcessor.getFunctionName(),
+                kafkaNodeProcessor.getValue());
+          case "kafkaPartitionId":
+            return createScanSpecForPartition(kafkaNodeProcessor.getFunctionName(),
+                kafkaNodeProcessor.getValue());
+        }
+      }
+    }
+    return null; //Return null, do not pushdown
+  }
+
+
+  private List<KafkaPartitionScanSpec> createScanSpecForTimestamp(String functionName,
+                                                                  Long fieldValue) {
+    List<KafkaPartitionScanSpec> scanSpec = Lists.newArrayList();
+    Map<TopicPartition, Long> timesValMap = Maps.newHashMap();
+    ImmutableSet<TopicPartition> topicPartitions = fullScanSpec.keySet();
+
+    for(TopicPartition partitions : topicPartitions) {
+      timesValMap.put(partitions, functionName.equals("greater_than") ? fieldValue+1 : fieldValue);
+    }
+
+    Map<TopicPartition, OffsetAndTimestamp> offsetAndTimestamp = kafkaConsumer.offsetsForTimes(timesValMap);
+
+    for(TopicPartition tp : topicPartitions) {
+      OffsetAndTimestamp value = offsetAndTimestamp.get(tp);
+      //OffsetAndTimestamp is null if there is no offset greater or equal to requested timestamp
+      if(value == null) {
+        scanSpec.add(
+            new KafkaPartitionScanSpec(tp.topic(), tp.partition(),
+                fullScanSpec.get(tp).getEndOffset(), fullScanSpec.get(tp).getEndOffset()));
+      } else {
+        scanSpec.add(
+            new KafkaPartitionScanSpec(tp.topic(), tp.partition(),
+                value.offset(), fullScanSpec.get(tp).getEndOffset()));
+      }
+    }
+
+    return scanSpec;
+  }
+
+  private List<KafkaPartitionScanSpec> createScanSpecForOffset(String functionName,
+                                                               Long fieldValue) {
+    List<KafkaPartitionScanSpec> scanSpec = Lists.newArrayList();
+    ImmutableSet<TopicPartition> topicPartitions = fullScanSpec.keySet();
+
+    /*
+    We should handle the case where the specified offset does not exist in the current context,
+    i.e., fieldValue < startOffset or fieldValue > endOffset in a particular topic-partition.
+    Else, KafkaConsumer.poll will throw "TimeoutException".
+    */
+
+    switch (functionName) {
+      case "equal":
+        for(TopicPartition tp : topicPartitions) {
+          if(fieldValue < fullScanSpec.get(tp).getStartOffset()) {
+            //Offset does not exist
+            scanSpec.add(
+                new KafkaPartitionScanSpec(tp.topic(), tp.partition(),
+                    fullScanSpec.get(tp).getEndOffset(), fullScanSpec.get(tp).getEndOffset()));
+          } else {
+            long val = Math.min(fieldValue, fullScanSpec.get(tp).getEndOffset());
+            long nextVal = Math.min(val+1, fullScanSpec.get(tp).getEndOffset());
+            scanSpec.add(new KafkaPartitionScanSpec(tp.topic(), tp.partition(), val, nextVal));
+          }
+        }
+        break;
+      case "greater_than_or_equal_to":
+        for(TopicPartition tp : topicPartitions) {
+          //Ensure scan range is between startOffset and endOffset,
+          long val = bindOffsetToRange(tp, fieldValue);
+          scanSpec.add(
+              new KafkaPartitionScanSpec(tp.topic(), tp.partition(), val,
+                  fullScanSpec.get(tp).getEndOffset()));
+        }
+        break;
+      case "greater_than":
+        for(TopicPartition tp : topicPartitions) {
+          //Ensure scan range is between startOffset and endOffset,
+          long val = bindOffsetToRange(tp, fieldValue+1);
+          scanSpec.add(
+              new KafkaPartitionScanSpec(tp.topic(), tp.partition(),
+                  val, fullScanSpec.get(tp).getEndOffset()));
+        }
+        break;
+      case "less_than_or_equal_to":
+        for(TopicPartition tp : topicPartitions) {
+          //Ensure scan range is between startOffset and endOffset,
+          long val = bindOffsetToRange(tp, fieldValue+1);
+
+          scanSpec.add(
+              new KafkaPartitionScanSpec(tp.topic(), tp.partition(),
+                  fullScanSpec.get(tp).getStartOffset(), val));
+        }
+        break;
+      case "less_than":
+        for(TopicPartition tp : topicPartitions) {
+          //Ensure scan range is between startOffset and endOffset,
+          long val = bindOffsetToRange(tp, fieldValue);
+
+          scanSpec.add(
+              new KafkaPartitionScanSpec(tp.topic(), tp.partition(),
+                  fullScanSpec.get(tp).getStartOffset(), val));
+        }
+        break;
+    }
+    return scanSpec;
+  }
+
+  private List<KafkaPartitionScanSpec> createScanSpecForPartition(String functionName,
+                                                                  Long fieldValue) {
+    List<KafkaPartitionScanSpec> scanSpecList = Lists.newArrayList();
+    ImmutableSet<TopicPartition> topicPartitions = fullScanSpec.keySet();
+
+    switch (functionName) {
+      case "equal":
+        for(TopicPartition tp : topicPartitions) {
+          if(tp.partition() == fieldValue) {
+            scanSpecList.add(
+                new KafkaPartitionScanSpec(tp.topic(), tp.partition(),
+                    fullScanSpec.get(tp).getStartOffset(),
+                    fullScanSpec.get(tp).getEndOffset()));
+          }
+        }
+        break;
+      case "not_equal":
+        for(TopicPartition tp : topicPartitions) {
+          if(tp.partition() != fieldValue) {
+            scanSpecList.add(
+                new KafkaPartitionScanSpec(tp.topic(), tp.partition(),
+                    fullScanSpec.get(tp).getStartOffset(),
+                    fullScanSpec.get(tp).getEndOffset()));
+          }
+        }
+        break;
+      case "greater_than_or_equal_to":
+        for(TopicPartition tp : topicPartitions) {
+          if(tp.partition() >= fieldValue) {
+            scanSpecList.add(
+                new KafkaPartitionScanSpec(tp.topic(), tp.partition(),
+                    fullScanSpec.get(tp).getStartOffset(),
+                    fullScanSpec.get(tp).getEndOffset()));
+          }
+        }
+        break;
+      case "greater_than":
+        for(TopicPartition tp : topicPartitions) {
+          if(tp.partition() > fieldValue) {
+            scanSpecList.add(
+                new KafkaPartitionScanSpec(tp.topic(), tp.partition(),
+                    fullScanSpec.get(tp).getStartOffset(),
+                    fullScanSpec.get(tp).getEndOffset()));
+          }
+        }
+        break;
+      case "less_than_or_equal_to":
+        for(TopicPartition tp : topicPartitions) {
+          if(tp.partition() <= fieldValue) {
+            scanSpecList.add(
+                new KafkaPartitionScanSpec(tp.topic(), tp.partition(),
+                    fullScanSpec.get(tp).getStartOffset(),
+                    fullScanSpec.get(tp).getEndOffset()));
+          }
+        }
+        break;
+      case "less_than":
+        for(TopicPartition tp : topicPartitions) {
+          if(tp.partition() < fieldValue) {
+            scanSpecList.add(
+                new KafkaPartitionScanSpec(tp.topic(), tp.partition(),
+                    fullScanSpec.get(tp).getStartOffset(),
+                    fullScanSpec.get(tp).getEndOffset()));
+          }
+        }
+        break;
+    }
+    return scanSpecList;
+  }
+
+  void close() {
+    kafkaConsumer.close(CLOSE_TIMEOUT_MS, TimeUnit.MILLISECONDS);
+  }
+
+  private long bindOffsetToRange(TopicPartition tp, long offset) {
+    return Math.max(fullScanSpec.get(tp).getStartOffset(), Math.min(offset, fullScanSpec.get(tp).getEndOffset()));
+  }
+}
diff --git a/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaPushDownFilterIntoScan.java b/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaPushDownFilterIntoScan.java
new file mode 100644
index 0000000000..bf11f852c9
--- /dev/null
+++ b/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaPushDownFilterIntoScan.java
@@ -0,0 +1,81 @@
+/*
+ * 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.kafka;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.rex.RexNode;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.planner.logical.DrillOptiq;
+import org.apache.drill.exec.planner.logical.DrillParseContext;
+import org.apache.drill.exec.planner.logical.RelOptHelper;
+import org.apache.drill.exec.planner.physical.FilterPrel;
+import org.apache.drill.exec.planner.physical.PrelUtil;
+import org.apache.drill.exec.planner.physical.ScanPrel;
+import org.apache.drill.exec.store.StoragePluginOptimizerRule;
+import java.util.List;
+
+public class KafkaPushDownFilterIntoScan extends StoragePluginOptimizerRule {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(KafkaPushDownFilterIntoScan.class);
+
+  public static final StoragePluginOptimizerRule INSTANCE =
+      new KafkaPushDownFilterIntoScan(RelOptHelper.some(FilterPrel.class, RelOptHelper.any(ScanPrel.class)),
+          "KafkaPushFilterIntoScan:Filter_On_Scan");
+
+  private KafkaPushDownFilterIntoScan(RelOptRuleOperand operand, String description) {
+    super(operand, description);
+  }
+
+  @Override
+  public void onMatch(RelOptRuleCall call) {
+    final ScanPrel scan = call.rel(1);
+    final FilterPrel filter = call.rel(0);
+    final RexNode condition = filter.getCondition();
+
+    LogicalExpression conditionExp =
+        DrillOptiq.toDrill(new DrillParseContext(PrelUtil.getPlannerSettings(call.getPlanner())), scan, condition);
+
+    KafkaGroupScan groupScan = (KafkaGroupScan) scan.getGroupScan();
+    logger.info("Partitions ScanSpec before pushdown: " + groupScan.getPartitionScanSpecList());
+    KafkaPartitionScanSpecBuilder builder = new KafkaPartitionScanSpecBuilder(groupScan, conditionExp);
+    List<KafkaPartitionScanSpec> newScanSpec = null;
+    newScanSpec = builder.parseTree();
+    builder.close(); //Close consumer
+
+    //No pushdown
+    if(newScanSpec == null) {
+      return;
+    }
+
+    logger.info("Partitions ScanSpec after pushdown: " + newScanSpec);
+    GroupScan newGroupScan = groupScan.cloneWithNewSpec(newScanSpec);
+    final ScanPrel newScanPrel = ScanPrel.create(scan, filter.getTraitSet(), newGroupScan, scan.getRowType());
+    call.transformTo(filter.copy(filter.getTraitSet(), ImmutableList.of(newScanPrel)));
+  }
+
+  @Override
+  public boolean matches(RelOptRuleCall call) {
+    final ScanPrel scan = (ScanPrel) call.rel(1);
+    if (scan.getGroupScan() instanceof KafkaGroupScan) {
+      return super.matches(call);
+    }
+    return false;
+  }
+}
diff --git a/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaRecordReader.java b/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaRecordReader.java
index b1cf9cd9b6..a0fc1f1117 100644
--- a/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaRecordReader.java
+++ b/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaRecordReader.java
@@ -31,7 +31,6 @@
 import org.apache.drill.exec.physical.impl.OutputMutator;
 import org.apache.drill.exec.server.options.OptionManager;
 import org.apache.drill.exec.store.AbstractRecordReader;
-import org.apache.drill.exec.store.kafka.KafkaSubScan.KafkaSubScanSpec;
 import org.apache.drill.exec.store.kafka.decoders.MessageReader;
 import org.apache.drill.exec.store.kafka.decoders.MessageReaderFactory;
 import org.apache.drill.exec.vector.complex.impl.VectorContainerWriter;
@@ -52,7 +51,7 @@
 
   private final boolean unionEnabled;
   private final KafkaStoragePlugin plugin;
-  private final KafkaSubScanSpec subScanSpec;
+  private final KafkaPartitionScanSpec subScanSpec;
   private final long kafkaPollTimeOut;
 
   private long currentOffset;
@@ -62,7 +61,7 @@
   private final boolean readNumbersAsDouble;
   private final String kafkaMsgReader;
 
-  public KafkaRecordReader(KafkaSubScan.KafkaSubScanSpec subScanSpec, List<SchemaPath> projectedColumns,
+  public KafkaRecordReader(KafkaPartitionScanSpec subScanSpec, List<SchemaPath> projectedColumns,
       FragmentContext context, KafkaStoragePlugin plugin) {
     setColumns(projectedColumns);
     final OptionManager optionManager = context.getOptions();
diff --git a/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaScanBatchCreator.java b/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaScanBatchCreator.java
index 9bedbd5c07..ae78d8c37c 100644
--- a/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaScanBatchCreator.java
+++ b/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaScanBatchCreator.java
@@ -44,7 +44,7 @@ public CloseableRecordBatch getBatch(ExecutorFragmentContext context, KafkaSubSc
     List<SchemaPath> columns = subScan.getColumns() != null ? subScan.getColumns() : GroupScan.ALL_COLUMNS;
 
     List<RecordReader> readers = new LinkedList<>();
-    for (KafkaSubScan.KafkaSubScanSpec scanSpec : subScan.getPartitionSubScanSpecList()) {
+    for (KafkaPartitionScanSpec scanSpec : subScan.getPartitionSubScanSpecList()) {
       readers.add(new KafkaRecordReader(scanSpec, columns, context, subScan.getKafkaStoragePlugin()));
     }
 
diff --git a/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaStoragePlugin.java b/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaStoragePlugin.java
index 157e3673ea..4ca91ec473 100644
--- a/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaStoragePlugin.java
+++ b/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaStoragePlugin.java
@@ -71,7 +71,7 @@ public void registerSchemas(SchemaConfig schemaConfig, SchemaPlus parent) throws
 
   @Override
   public Set<StoragePluginOptimizerRule> getPhysicalOptimizerRules(OptimizerRulesContext optimizerRulesContext) {
-    return ImmutableSet.of();
+    return ImmutableSet.of(KafkaPushDownFilterIntoScan.INSTANCE);
   }
 
   @Override
diff --git a/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaSubScan.java b/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaSubScan.java
index 468f766a96..d62faa60cc 100644
--- a/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaSubScan.java
+++ b/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaSubScan.java
@@ -43,14 +43,14 @@
 
   private final KafkaStoragePlugin kafkaStoragePlugin;
   private final List<SchemaPath> columns;
-  private final List<KafkaSubScanSpec> partitionSubScanSpecList;
+  private final List<KafkaPartitionScanSpec> partitionSubScanSpecList;
 
   @JsonCreator
   public KafkaSubScan(@JacksonInject StoragePluginRegistry registry,
                       @JsonProperty("userName") String userName,
                       @JsonProperty("kafkaStoragePluginConfig") KafkaStoragePluginConfig kafkaStoragePluginConfig,
                       @JsonProperty("columns") List<SchemaPath> columns,
-                      @JsonProperty("partitionSubScanSpecList") LinkedList<KafkaSubScanSpec> partitionSubScanSpecList)
+                      @JsonProperty("partitionSubScanSpecList") LinkedList<KafkaPartitionScanSpec> partitionSubScanSpecList)
       throws ExecutionSetupException {
     this(userName,
         (KafkaStoragePlugin) registry.getPlugin(kafkaStoragePluginConfig),
@@ -61,7 +61,7 @@ public KafkaSubScan(@JacksonInject StoragePluginRegistry registry,
   public KafkaSubScan(String userName,
                       KafkaStoragePlugin kafkaStoragePlugin,
                       List<SchemaPath> columns,
-                      List<KafkaSubScanSpec> partitionSubScanSpecList) {
+                      List<KafkaPartitionScanSpec> partitionSubScanSpecList) {
     super(userName);
     this.kafkaStoragePlugin = kafkaStoragePlugin;
     this.columns = columns;
@@ -95,7 +95,7 @@ public KafkaStoragePluginConfig getKafkaStoragePluginConfig() {
   }
 
   @JsonProperty
-  public List<KafkaSubScanSpec> getPartitionSubScanSpecList() {
+  public List<KafkaPartitionScanSpec> getPartitionSubScanSpecList() {
     return partitionSubScanSpecList;
   }
 
@@ -108,68 +108,4 @@ public KafkaStoragePlugin getKafkaStoragePlugin() {
   public int getOperatorType() {
     return CoreOperatorType.KAFKA_SUB_SCAN_VALUE;
   }
-
-  public static class KafkaSubScanSpec {
-    protected String topicName;
-    protected int partitionId;
-    protected long startOffset;
-    protected long endOffset;
-
-    @JsonCreator
-    public KafkaSubScanSpec(@JsonProperty("topicName") String topicName, @JsonProperty("partitionId") int partitionId,
-        @JsonProperty("startOffset") long startOffset, @JsonProperty("endOffset") long endOffset) {
-      this.topicName = topicName;
-      this.partitionId = partitionId;
-      this.startOffset = startOffset;
-      this.endOffset = endOffset;
-    }
-
-    KafkaSubScanSpec() {
-
-    }
-
-    public String getTopicName() {
-      return topicName;
-    }
-
-    public int getPartitionId() {
-      return partitionId;
-    }
-
-    public long getStartOffset() {
-      return startOffset;
-    }
-
-    public long getEndOffset() {
-      return endOffset;
-    }
-
-    public KafkaSubScanSpec setTopicName(String topicName) {
-      this.topicName = topicName;
-      return this;
-    }
-
-    public KafkaSubScanSpec setPartitionId(int partitionId) {
-      this.partitionId = partitionId;
-      return this;
-    }
-
-    public KafkaSubScanSpec setStartOffset(long startOffset) {
-      this.startOffset = startOffset;
-      return this;
-    }
-
-    public KafkaSubScanSpec setEndOffset(long endOffset) {
-      this.endOffset = endOffset;
-      return this;
-    }
-
-    @Override
-    public String toString() {
-      return "KafkaSubScanSpec [topicName=" + topicName + ", partitionId=" + partitionId + ", startOffset="
-          + startOffset + ", endOffset=" + endOffset + "]";
-    }
-
-  }
-
 }
diff --git a/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/MessageIterator.java b/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/MessageIterator.java
index 3afb1b8d45..1c814f6843 100644
--- a/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/MessageIterator.java
+++ b/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/MessageIterator.java
@@ -23,7 +23,6 @@
 
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.ExecConstants;
-import org.apache.drill.exec.store.kafka.KafkaSubScan.KafkaSubScanSpec;
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.clients.consumer.ConsumerRecords;
 import org.apache.kafka.clients.consumer.KafkaConsumer;
@@ -46,7 +45,7 @@
   private final long kafkaPollTimeOut;
   private final long endOffset;
 
-  public MessageIterator(final KafkaConsumer<byte[], byte[]> kafkaConsumer, final KafkaSubScanSpec subScanSpec,
+  public MessageIterator(final KafkaConsumer<byte[], byte[]> kafkaConsumer, final KafkaPartitionScanSpec subScanSpec,
       final long kafkaPollTimeOut) {
     this.kafkaConsumer = kafkaConsumer;
     this.kafkaPollTimeOut = kafkaPollTimeOut;
diff --git a/contrib/storage-kafka/src/main/resources/bootstrap-storage-plugins.json b/contrib/storage-kafka/src/main/resources/bootstrap-storage-plugins.json
index 406c030608..18a1df5641 100644
--- a/contrib/storage-kafka/src/main/resources/bootstrap-storage-plugins.json
+++ b/contrib/storage-kafka/src/main/resources/bootstrap-storage-plugins.json
@@ -2,8 +2,8 @@
   "storage":{
     kafka : {
       type:"kafka",
-      enabled: false,
-      kafkaConsumerProps: {"bootstrap.servers":"localhost:9092", "group.id" : "drill-consumer"}
+      kafkaConsumerProps: {"bootstrap.servers":"localhost:9092", "group.id" : "drill-consumer"},
+      enabled: false
     }
   }
 }
diff --git a/contrib/storage-kafka/src/test/java/org/apache/drill/exec/store/kafka/KafkaFilterPushdownTest.java b/contrib/storage-kafka/src/test/java/org/apache/drill/exec/store/kafka/KafkaFilterPushdownTest.java
new file mode 100644
index 0000000000..7be0ec3941
--- /dev/null
+++ b/contrib/storage-kafka/src/test/java/org/apache/drill/exec/store/kafka/KafkaFilterPushdownTest.java
@@ -0,0 +1,359 @@
+/*
+ * 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.kafka;
+
+import org.apache.drill.categories.KafkaStorageTest;
+import org.apache.drill.categories.SlowTest;
+import org.apache.kafka.common.serialization.StringSerializer;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import static org.apache.drill.exec.store.kafka.TestKafkaSuit.NUM_JSON_MSG;
+import static org.apache.drill.exec.store.kafka.TestKafkaSuit.embeddedKafkaCluster;
+
+@Category({KafkaStorageTest.class, SlowTest.class})
+public class KafkaFilterPushdownTest extends KafkaTestBase {
+  private static final int NUM_PARTITIONS = 5;
+  private static final String expectedSubStr = "    \"kafkaScanSpec\" : {\n" +
+                                                   "      \"topicName\" : \"drill-pushdown-topic\"\n" +
+                                                   "    },\n" +
+                                                   "    \"cost\" : %s.0";
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    TestKafkaSuit.createTopicHelper(TestQueryConstants.JSON_PUSHDOWN_TOPIC, NUM_PARTITIONS);
+    KafkaMessageGenerator generator = new KafkaMessageGenerator(embeddedKafkaCluster.getKafkaBrokerList(),
+        StringSerializer.class);
+    generator.populateJsonMsgWithTimestamps(TestQueryConstants.JSON_PUSHDOWN_TOPIC, NUM_JSON_MSG);
+  }
+
+  /**
+   * Test filter pushdown with condition on kafkaMsgOffset.
+   * @throws Exception
+   */
+  @Test
+  public void testPushdownOnOffset() throws Exception {
+    final String predicate1 = "kafkaMsgOffset > 4";
+    final String predicate2 = "kafkaMsgOffset < 6";
+    final int expectedRowCount = 5; //1 * NUM_PARTITIONS
+
+    final String queryString = String.format(TestQueryConstants.QUERY_TEMPLATE_AND,
+        TestQueryConstants.JSON_PUSHDOWN_TOPIC, predicate1, predicate2);
+
+    runKafkaSQLVerifyCount(queryString, expectedRowCount);
+    testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCount));
+  }
+
+  /**
+   * Test filter pushdown with condition on kafkaPartitionId.
+   * @throws Exception
+   */
+  @Test
+  public void testPushdownOnPartition() throws Exception {
+    final String predicate = "kafkaPartitionId = 1";
+    final int expectedRowCount = NUM_JSON_MSG;
+
+    final String queryString = String.format(TestQueryConstants.QUERY_TEMPLATE_BASIC,
+        TestQueryConstants.JSON_PUSHDOWN_TOPIC, predicate);
+
+    runKafkaSQLVerifyCount(queryString, expectedRowCount);
+    testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCount));
+  }
+
+  /**
+   * Test filter pushdown with condition on kafkaPartitionId.
+   * @throws Exception
+   */
+  @Test
+  public void testPushdownOnTimestamp() throws Exception {
+    final String predicate = "kafkaMsgTimestamp > 6";
+    final int expectedRowCount = 20;
+
+    final String queryString = String.format(TestQueryConstants.QUERY_TEMPLATE_BASIC,
+        TestQueryConstants.JSON_PUSHDOWN_TOPIC, predicate);
+
+    runKafkaSQLVerifyCount(queryString,expectedRowCount);
+    testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCount));
+  }
+
+  /**
+   * Test filter pushdown when timestamp is not ordered.
+   * @throws Exception
+   */
+  @Test
+  public void testPushdownUnorderedTimestamp() throws Exception {
+    final String predicate = "kafkaMsgTimestamp = 1";
+    final int expectedRowInPlan = 50;
+    final int expectedRowCount = 5;
+
+    final String queryString = String.format(TestQueryConstants.QUERY_TEMPLATE_BASIC,
+        TestQueryConstants.JSON_PUSHDOWN_TOPIC, predicate);
+
+    runKafkaSQLVerifyCount(queryString,expectedRowCount);
+    testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowInPlan));
+  }
+
+  /**
+   * Test filter pushdown when timestamp value specified does not exist.
+   * @throws Exception
+   */
+  @Test
+  public void testPushdownWhenTimestampDoesNotExist() throws Exception {
+    final String predicate = "kafkaMsgTimestamp = 20"; //20 does not exist
+    final int expectedRowCount = 0;
+
+    final String queryString = String.format(TestQueryConstants.QUERY_TEMPLATE_BASIC,
+        TestQueryConstants.JSON_PUSHDOWN_TOPIC, predicate);
+
+    runKafkaSQLVerifyCount(queryString,expectedRowCount);
+    testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCount));
+  }
+
+  /**
+   * Test filter pushdown when partition value specified does not exist.
+   * @throws Exception
+   */
+  @Test
+  public void testPushdownWhenPartitionDoesNotExist() throws Exception {
+    final String predicate = "kafkaPartitionId = 100"; //100 does not exist
+    final int expectedRowCount = 0;
+
+    final String queryString = String.format(TestQueryConstants.QUERY_TEMPLATE_BASIC,
+        TestQueryConstants.JSON_PUSHDOWN_TOPIC, predicate);
+
+    runKafkaSQLVerifyCount(queryString,expectedRowCount);
+    testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCount));
+  }
+
+  /**
+   * Test filter pushdown when timestamp exist but partition does not exist.
+   * @throws Exception
+   */
+  @Test
+  public void testPushdownForEmptyScanSpec() throws Exception {
+    final String predicate1 = "kafkaMsgTimestamp > 6";
+    final String predicate2 = "kafkaPartitionId = 100";
+    final int expectedRowCount = 0;
+
+    final String queryString = String.format(TestQueryConstants.QUERY_TEMPLATE_AND,
+        TestQueryConstants.JSON_PUSHDOWN_TOPIC, predicate1, predicate2);
+
+    runKafkaSQLVerifyCount(queryString,expectedRowCount);
+    testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCount));
+  }
+
+  /**
+   * Test filter pushdown on kafkaMsgOffset with boundary conditions.
+   * In every case, the number of records returned is 0.
+   * @throws Exception
+   */
+  @Test
+  public void testPushdownOffsetNoRecordsReturnedWithBoundaryConditions() throws Exception {
+    final int expectedRowCount = 0;
+
+    //"equal" such that value = endOffset
+    String queryString = String.format(TestQueryConstants.QUERY_TEMPLATE_BASIC,
+        TestQueryConstants.JSON_PUSHDOWN_TOPIC, "kafkaMsgOffset = 10");
+
+    runKafkaSQLVerifyCount(queryString,expectedRowCount);
+    testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCount));
+
+    //"equal" such that value < startOffset
+    queryString = String.format(TestQueryConstants.QUERY_TEMPLATE_BASIC,
+        TestQueryConstants.JSON_PUSHDOWN_TOPIC, "kafkaMsgOffset = -1");
+
+    runKafkaSQLVerifyCount(queryString,expectedRowCount);
+    testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCount));
+
+    //"greater_than" such that value = endOffset-1
+    queryString = String.format(TestQueryConstants.QUERY_TEMPLATE_BASIC,
+        TestQueryConstants.JSON_PUSHDOWN_TOPIC, "kafkaMsgOffset > 9");
+
+    runKafkaSQLVerifyCount(queryString,expectedRowCount);
+    testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCount));
+
+    //"greater_than_or_equal" such that value = endOffset
+    queryString = String.format(TestQueryConstants.QUERY_TEMPLATE_BASIC,
+        TestQueryConstants.JSON_PUSHDOWN_TOPIC, "kafkaMsgOffset >= 10");
+
+    runKafkaSQLVerifyCount(queryString,expectedRowCount);
+    testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCount));
+
+    //"less_than" such that value = startOffset
+    queryString = String.format(TestQueryConstants.QUERY_TEMPLATE_BASIC,
+        TestQueryConstants.JSON_PUSHDOWN_TOPIC, "kafkaMsgOffset < 0");
+
+    runKafkaSQLVerifyCount(queryString,expectedRowCount);
+    testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCount));
+
+    //"less_than_or_equal" such that value < startOffset
+    queryString = String.format(TestQueryConstants.QUERY_TEMPLATE_BASIC,
+        TestQueryConstants.JSON_PUSHDOWN_TOPIC, "kafkaMsgOffset <= -1");
+
+    runKafkaSQLVerifyCount(queryString,expectedRowCount);
+    testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCount));
+  }
+
+  /**
+   * Test filter pushdown on kafkaMsgOffset with boundary conditions.
+   * In every case, the number of records returned is 5 (1 per topic-partition).
+   * @throws Exception
+   */
+  @Test
+  public void testPushdownOffsetOneRecordReturnedWithBoundaryConditions() throws Exception {
+    final int expectedRowCount = 5;
+
+    //"equal" such that value = endOffset-1
+    String queryString = String.format(TestQueryConstants.QUERY_TEMPLATE_BASIC,
+        TestQueryConstants.JSON_PUSHDOWN_TOPIC, "kafkaMsgOffset = 9");
+
+    runKafkaSQLVerifyCount(queryString, expectedRowCount);
+    testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCount));
+
+    //"greater_than" such that value = endOffset-2
+    queryString = String.format(TestQueryConstants.QUERY_TEMPLATE_BASIC,
+        TestQueryConstants.JSON_PUSHDOWN_TOPIC, "kafkaMsgOffset > 8");
+
+    runKafkaSQLVerifyCount(queryString,expectedRowCount);
+    testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCount));
+
+    //"greater_than_or_equal" such that value = endOffset-1
+    queryString = String.format(TestQueryConstants.QUERY_TEMPLATE_BASIC,
+        TestQueryConstants.JSON_PUSHDOWN_TOPIC, "kafkaMsgOffset >= 9");
+
+    runKafkaSQLVerifyCount(queryString,expectedRowCount);
+    testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCount));
+  }
+
+  /**
+   * Test filter pushdown with OR.
+   * Pushdown is possible if all the predicates are on metadata fields.
+   * @throws Exception
+   */
+  @Test
+  public void testPushdownWithOr() throws Exception {
+    final String predicate1 = "kafkaMsgTimestamp > 6";
+    final String predicate2 = "kafkaPartitionId = 1";
+    final int expectedRowCount = 26;
+
+    final String queryString = String.format(TestQueryConstants.QUERY_TEMPLATE_OR,
+        TestQueryConstants.JSON_PUSHDOWN_TOPIC, predicate1, predicate2);
+
+    runKafkaSQLVerifyCount(queryString,expectedRowCount);
+    testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCount));
+  }
+
+  /**
+   * Test filter pushdown with OR on kafkaMsgTimestamp and kafkaMsgOffset.
+   * @throws Exception
+   */
+  @Test
+  public void testPushdownWithOr1() throws Exception {
+    final String predicate1 = "kafkaMsgTimestamp = 6";
+    final String predicate2 = "kafkaMsgOffset = 6";
+    final int expectedRowInPlan = 25; //startOff=5, endOff=9
+    final int expectedRowCount = 10;
+
+    final String queryString = String.format(TestQueryConstants.QUERY_TEMPLATE_OR,
+        TestQueryConstants.JSON_PUSHDOWN_TOPIC, predicate1, predicate2);
+
+    runKafkaSQLVerifyCount(queryString,expectedRowCount);
+    testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowInPlan));
+  }
+
+  /**
+   * Test pushdown for a combination of AND and OR.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testPushdownWithAndOrCombo() throws Exception {
+    final String predicate1 = "kafkaMsgTimestamp > 6";
+    final String predicate2 = "kafkaPartitionId = 1";
+    final String predicate3 = "kafkaPartitionId = 2";
+    final int expectedRowCount = 8;
+
+    final String queryString = String.format(TestQueryConstants.QUERY_TEMPLATE_AND_OR_PATTERN_1,
+        TestQueryConstants.JSON_PUSHDOWN_TOPIC, predicate1, predicate2, predicate3);
+
+    runKafkaSQLVerifyCount(queryString,expectedRowCount);
+    testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCount));
+  }
+
+  /**
+   * Test pushdown for a combination of AND and OR.
+   * @throws Exception
+   */
+  @Test
+  public void testPushdownWithAndOrCombo2() throws Exception {
+    final String predicate1 = "kafkaMsgTimestamp = 6";
+    final String predicate2 = "kafkaMsgOffset = 6";
+    final String predicate3 = "kafkaPartitionId = 1";
+    final String predicate4 = "kafkaPartitionId = 2";
+    final int expectedRowCountInPlan = 10; //startOff=5, endOff=9 for 2 partitions
+    final int expectedRowCount = 4;
+
+    final String queryString = String.format(TestQueryConstants.QUERY_TEMPLATE_AND_OR_PATTERN_3,
+        TestQueryConstants.JSON_PUSHDOWN_TOPIC, predicate1, predicate2, predicate3, predicate4);
+
+    runKafkaSQLVerifyCount(queryString,expectedRowCount);
+    testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCountInPlan));
+  }
+
+  /**
+   * Test pushdown for predicate1 AND predicate2.
+   * Where predicate1 is on metadata field and and predicate2 is on user fields.
+   * @throws Exception
+   */
+  @Test
+  public void testPushdownTimestampWithNonMetaField() throws Exception {
+    final String predicate1 = "kafkaMsgTimestamp > 6";
+    final String predicate2 = "boolKey = true";
+    final int expectedRowCountInPlan = 20; //startOff=5, endOff=9
+    final int expectedRowCount = 10;
+
+    final String queryString = String.format(TestQueryConstants.QUERY_TEMPLATE_AND,
+        TestQueryConstants.JSON_PUSHDOWN_TOPIC, predicate1, predicate2);
+
+    runKafkaSQLVerifyCount(queryString,expectedRowCount);
+    testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCountInPlan));
+  }
+
+  /**
+   * Tests that pushdown does not happen for predicates such as
+   * non-metadata-field = val1 OR (kafkaMsgTimestamp > val2 AND kafkaMsgTimestamp < val4)
+   * @throws Exception
+   */
+  @Test
+  public void testNoPushdownOfOffsetWithNonMetadataField() throws Exception {
+    final String predicate1 = "boolKey = true";
+    final String predicate2 = "kafkaMsgTimestamp > 6";
+    final String predicate3 = "kafkaMsgTimestamp < 9";
+    final int expectedRowCountInPlan = 50; //no pushdown
+    final int expectedRowCount = 30;
+
+    final String queryString = String.format(TestQueryConstants.QUERY_TEMPLATE_AND_OR_PATTERN_2,
+        TestQueryConstants.JSON_PUSHDOWN_TOPIC, predicate1, predicate2, predicate3);
+
+    runKafkaSQLVerifyCount(queryString,expectedRowCount);
+    testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCountInPlan));
+  }
+
+}
diff --git a/contrib/storage-kafka/src/test/java/org/apache/drill/exec/store/kafka/KafkaMessageGenerator.java b/contrib/storage-kafka/src/test/java/org/apache/drill/exec/store/kafka/KafkaMessageGenerator.java
index 1931898a53..32d9ff10a4 100644
--- a/contrib/storage-kafka/src/test/java/org/apache/drill/exec/store/kafka/KafkaMessageGenerator.java
+++ b/contrib/storage-kafka/src/test/java/org/apache/drill/exec/store/kafka/KafkaMessageGenerator.java
@@ -35,6 +35,7 @@
 import org.apache.kafka.clients.producer.ProducerConfig;
 import org.apache.kafka.clients.producer.ProducerRecord;
 import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.PartitionInfo;
 import org.apache.kafka.common.serialization.StringSerializer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -131,4 +132,37 @@ public void populateJsonMsgIntoKafka(String topic, int numMsg) throws Interrupte
     }
   }
 
+  public void populateJsonMsgWithTimestamps(String topic, int numMsg) {
+    KafkaProducer<String, String> producer = null;
+    Random rand = new Random();
+    try {
+      producer = new KafkaProducer<String, String>(producerProperties);
+      int halfCount = numMsg / 2;
+
+      for(PartitionInfo tpInfo : producer.partitionsFor(topic)) {
+        for (int i = 1; i <= numMsg; ++i) {
+          JsonObject object = new JsonObject();
+          object.addProperty("stringKey", UUID.randomUUID().toString());
+          object.addProperty("intKey", numMsg - i);
+          object.addProperty("boolKey", i % 2 == 0);
+
+          long timestamp = i < halfCount ? (halfCount - i) : i;
+          ProducerRecord<String, String> message =
+              new ProducerRecord<String, String>(tpInfo.topic(), tpInfo.partition(), timestamp, "key"+i, object.toString());
+          logger.info("Publishing message : {}", message);
+          Future<RecordMetadata> future = producer.send(message);
+          logger.info("Committed offset of the message : {}", future.get().offset());
+        }
+
+      }
+    } catch (Throwable th) {
+      logger.error(th.getMessage(), th);
+      throw new DrillRuntimeException(th.getMessage(), th);
+    } finally {
+      if (producer != null) {
+        producer.close();
+      }
+    }
+  }
+
 }
diff --git a/contrib/storage-kafka/src/test/java/org/apache/drill/exec/store/kafka/MessageIteratorTest.java b/contrib/storage-kafka/src/test/java/org/apache/drill/exec/store/kafka/MessageIteratorTest.java
index 4a155963fa..4347167576 100644
--- a/contrib/storage-kafka/src/test/java/org/apache/drill/exec/store/kafka/MessageIteratorTest.java
+++ b/contrib/storage-kafka/src/test/java/org/apache/drill/exec/store/kafka/MessageIteratorTest.java
@@ -25,7 +25,6 @@
 import org.apache.drill.categories.SlowTest;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType;
-import org.apache.drill.exec.store.kafka.KafkaSubScan.KafkaSubScanSpec;
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.clients.consumer.KafkaConsumer;
@@ -40,7 +39,7 @@
 public class MessageIteratorTest extends KafkaTestBase {
 
   private KafkaConsumer<byte[], byte[]> kafkaConsumer;
-  private KafkaSubScanSpec subScanSpec;
+  private KafkaPartitionScanSpec subScanSpec;
 
   @Before
   public void setUp() {
@@ -49,7 +48,7 @@ public void setUp() {
     consumerProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class);
     consumerProps.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, "4");
     kafkaConsumer = new KafkaConsumer<>(consumerProps);
-    subScanSpec = new KafkaSubScanSpec(TestQueryConstants.JSON_TOPIC, 0, 0, TestKafkaSuit.NUM_JSON_MSG);
+    subScanSpec = new KafkaPartitionScanSpec(TestQueryConstants.JSON_TOPIC, 0, 0, TestKafkaSuit.NUM_JSON_MSG);
   }
 
   @After
diff --git a/contrib/storage-kafka/src/test/java/org/apache/drill/exec/store/kafka/TestKafkaSuit.java b/contrib/storage-kafka/src/test/java/org/apache/drill/exec/store/kafka/TestKafkaSuit.java
index ed0174755a..ecf998e3ad 100644
--- a/contrib/storage-kafka/src/test/java/org/apache/drill/exec/store/kafka/TestKafkaSuit.java
+++ b/contrib/storage-kafka/src/test/java/org/apache/drill/exec/store/kafka/TestKafkaSuit.java
@@ -27,6 +27,7 @@
 import org.apache.drill.exec.ZookeeperTestUtil;
 import org.apache.drill.exec.store.kafka.cluster.EmbeddedKafkaCluster;
 import org.apache.drill.exec.store.kafka.decoders.MessageReaderFactoryTest;
+import org.apache.kafka.common.config.TopicConfig;
 import org.apache.kafka.common.serialization.StringSerializer;
 import org.apache.kafka.common.security.JaasUtils;
 
@@ -46,7 +47,8 @@
 
 @Category({KafkaStorageTest.class, SlowTest.class})
 @RunWith(Suite.class)
-@SuiteClasses({ KafkaQueriesTest.class, MessageIteratorTest.class, MessageReaderFactoryTest.class })
+@SuiteClasses({ KafkaQueriesTest.class, MessageIteratorTest.class, MessageReaderFactoryTest.class,
+    KafkaFilterPushdownTest.class })
 public class TestKafkaSuit {
   private static final Logger logger = LoggerFactory.getLogger(LoggerFactory.class);
   private static final String LOGIN_CONF_RESOURCE_PATHNAME = "login.conf";
@@ -106,4 +108,18 @@ public static void tearDownCluster() throws Exception {
     }
   }
 
+  public static void createTopicHelper(final String topicName, final int partitions) {
+
+    Properties topicProps = new Properties();
+    topicProps.put(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG, "CreateTime");
+    ZkUtils zkUtils = new ZkUtils(zkClient,
+        new ZkConnection(embeddedKafkaCluster.getZkServer().getConnectionString()), false);
+    AdminUtils.createTopic(zkUtils, topicName, partitions, 1,
+        topicProps, RackAwareMode.Disabled$.MODULE$);
+
+    org.apache.kafka.common.requests.MetadataResponse.TopicMetadata fetchTopicMetadataFromZk =
+        AdminUtils.fetchTopicMetadataFromZk(topicName, zkUtils);
+    logger.info("Topic Metadata: " + fetchTopicMetadataFromZk);
+  }
+
 }
diff --git a/contrib/storage-kafka/src/test/java/org/apache/drill/exec/store/kafka/TestQueryConstants.java b/contrib/storage-kafka/src/test/java/org/apache/drill/exec/store/kafka/TestQueryConstants.java
index 057af7eb9b..b3163adbcd 100644
--- a/contrib/storage-kafka/src/test/java/org/apache/drill/exec/store/kafka/TestQueryConstants.java
+++ b/contrib/storage-kafka/src/test/java/org/apache/drill/exec/store/kafka/TestQueryConstants.java
@@ -29,12 +29,24 @@
   int MAX_CLIENT_CONNECTIONS = 100;
 
   String JSON_TOPIC = "drill-json-topic";
+  String JSON_PUSHDOWN_TOPIC = "drill-pushdown-topic";
   String AVRO_TOPIC = "drill-avro-topic";
   String INVALID_TOPIC = "invalid-topic";
 
+  String KAFKA_MSG_TIMESTAMP_FIELD = "kafkaMsgTimestamp";
+  String KAFKA_PARTITION_ID_FIELD = "kafkaPartitionId";
+  String KAFKA_MSG_OFFSET_FIELD = "kafkaMsgOffset";
+
   // Queries
   String MSG_COUNT_QUERY = "select count(*) from kafka.`%s`";
   String MSG_SELECT_QUERY = "select * from kafka.`%s`";
   String MIN_OFFSET_QUERY = "select MIN(kafkaMsgOffset) as minOffset from kafka.`%s`";
   String MAX_OFFSET_QUERY = "select MAX(kafkaMsgOffset) as maxOffset from kafka.`%s`";
+
+  String QUERY_TEMPLATE_BASIC = "select * from kafka.`%s` where %s";
+  String QUERY_TEMPLATE_AND = "select * from kafka.`%s` where %s AND %s";
+  String QUERY_TEMPLATE_OR = "select * from kafka.`%s` where %s OR %s";
+  String QUERY_TEMPLATE_AND_OR_PATTERN_1 = "select * from kafka.`%s` where %s AND (%s OR %s)";
+  String QUERY_TEMPLATE_AND_OR_PATTERN_2 = "select * from kafka.`%s` where %s OR (%s AND %s)";
+  String QUERY_TEMPLATE_AND_OR_PATTERN_3 = "select * from kafka.`%s` where (%s OR %s) AND (%s OR %s)";
 }
diff --git a/contrib/storage-kudu/src/main/codegen/templates/KuduRecordWriter.java b/contrib/storage-kudu/src/main/codegen/templates/KuduRecordWriter.java
index 9de0cf25d0..e424c9df4a 100644
--- a/contrib/storage-kudu/src/main/codegen/templates/KuduRecordWriter.java
+++ b/contrib/storage-kudu/src/main/codegen/templates/KuduRecordWriter.java
@@ -17,23 +17,6 @@
  */
 <@pp.dropOutputFile />
 <@pp.changeOutputFile name="org/apache/drill/exec/store/kudu/KuduRecordWriter.java" />
-/**
- * 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.kudu;
 
@@ -101,8 +84,8 @@ public void setUp(PartialRow row) {
   <#list vv.types as type>
     <#list type.minor as minor>
       <#list vv.modes as mode>
-      
-        <#if mode.prefix == "Repeated" || 
+
+        <#if mode.prefix == "Repeated" ||
         minor.class == "TinyInt" ||
         minor.class == "UInt1" ||
         minor.class == "UInt2" ||
@@ -116,31 +99,31 @@ public void setUp(PartialRow row) {
         minor.class == "Decimal38Sparse" ||
         minor.class?contains("Interval")
         >
-        
+
         <#else>
           @Override
           public FieldConverter getNew${mode.prefix}${minor.class}Converter(int fieldId, String fieldName, FieldReader reader) {
             return new ${mode.prefix}${minor.class}KuduConverter(fieldId, fieldName, reader);
           }
-      
+
           public class ${mode.prefix}${minor.class}KuduConverter extends FieldConverter {
             private Nullable${minor.class}Holder holder = new Nullable${minor.class}Holder();
-      
+
             public ${mode.prefix}${minor.class}KuduConverter(int fieldId, String fieldName, FieldReader reader) {
               super(fieldId, fieldName, reader);
             }
-      
+
             @Override
             public void writeField() throws IOException {
-         
+
           <#if mode.prefix == "Nullable" >
             if (!reader.isSet()) {
               return;
             }
           </#if>
-          
+
             reader.read(holder);
-            
+
             <#if minor.class == "Float4">
               row.addFloat(fieldId, holder.value);
             <#elseif minor.class == "TimeStamp">
diff --git a/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduConnect.java b/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduConnect.java
index bf66037725..ad1e7958ab 100644
--- a/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduConnect.java
+++ b/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduConnect.java
@@ -35,7 +35,6 @@
 import org.apache.kudu.client.KuduTable;
 import org.apache.kudu.client.ListTablesResponse;
 import org.apache.kudu.client.PartialRow;
-import org.apache.kudu.client.RowResult;
 import org.apache.kudu.client.RowResultIterator;
 import org.apache.kudu.client.SessionConfiguration;
 import org.junit.experimental.categories.Category;
@@ -99,8 +98,7 @@ public static void createKuduTable(String tableName, int tablets, int replicas,
       while (scanner.hasMoreRows()) {
         RowResultIterator results = scanner.nextRows();
         while (results.hasNext()) {
-          RowResult result = results.next();
-          System.out.println(result.toStringLongFormat());
+          logger.debug(results.next().toString());
         }
       }
     }
diff --git a/contrib/storage-mongo/src/main/resources/bootstrap-storage-plugins.json b/contrib/storage-mongo/src/main/resources/bootstrap-storage-plugins.json
index b7d34f2fb3..9983596d35 100644
--- a/contrib/storage-mongo/src/main/resources/bootstrap-storage-plugins.json
+++ b/contrib/storage-mongo/src/main/resources/bootstrap-storage-plugins.json
@@ -2,8 +2,8 @@
   "storage":{
     mongo : {
       type:"mongo",
-      enabled: false,
-      connection:"mongodb://localhost:27017/"
+      connection:"mongodb://localhost:27017/",
+      enabled: false
     }
   }
 }
diff --git a/contrib/storage-opentsdb/pom.xml b/contrib/storage-opentsdb/pom.xml
index fdd35d0b9c..4090ff0f05 100644
--- a/contrib/storage-opentsdb/pom.xml
+++ b/contrib/storage-opentsdb/pom.xml
@@ -57,7 +57,7 @@
         <dependency>
             <groupId>com.github.tomakehurst</groupId>
             <artifactId>wiremock-standalone</artifactId>
-            <version>2.5.1</version>
+            <version>${wiremock.standalone.version}</version>
             <scope>test</scope>
         </dependency>
 
diff --git a/distribution/src/assemble/bin.xml b/distribution/src/assemble/bin.xml
index e044801f9b..712f3ec596 100644
--- a/distribution/src/assemble/bin.xml
+++ b/distribution/src/assemble/bin.xml
@@ -420,6 +420,11 @@
       <source>src/resources/drill-on-yarn-example.conf</source>
       <outputDirectory>conf</outputDirectory>
       <fileMode>0640</fileMode>
-   </file>
+    </file>
+    <file>
+      <source>src/resources/storage-plugins-override-example.conf</source>
+      <outputDirectory>conf</outputDirectory>
+      <fileMode>0640</fileMode>
+    </file>
   </files>
 </assembly>
diff --git a/distribution/src/resources/drill-env.sh b/distribution/src/resources/drill-env.sh
index 62384e7012..7b70172ea3 100755
--- a/distribution/src/resources/drill-env.sh
+++ b/distribution/src/resources/drill-env.sh
@@ -80,7 +80,7 @@
 
 # Location to place Drill logs. Set to $DRILL_HOME/log by default.
 
-#export DRILL_LOG_DIR=${DRILL_LOG_DIR:-$DRILL_HOME/conf}
+#export DRILL_LOG_DIR=${DRILL_LOG_DIR:-$DRILL_HOME/log}
 
 # Location to place the Drillbit pid file when running as a daemon using
 # drillbit.sh start.
diff --git a/distribution/src/resources/drill-override-example.conf b/distribution/src/resources/drill-override-example.conf
index fa2e395559..296cd8b60e 100644
--- a/distribution/src/resources/drill-override-example.conf
+++ b/distribution/src/resources/drill-override-example.conf
@@ -58,17 +58,10 @@ drill.exec: {
         batch.size: 4000
       },
       partition.column.label: "dir"
-    }
-  },
-  metrics : {
-    context: "drillbit",
-    jmx: {
-      enabled : true
     },
-    log: {
-      enabled : false,
-      interval : 60
-    }
+    # The action on the storage-plugins-override.conf after it's use.
+    # Possible values are "none" (default), "rename", "remove"
+    action_on_plugins_override_file: "none"
   },
   zk: {
 	connect: "localhost:2181",
@@ -252,6 +245,15 @@ drill.exec: {
     #ssl provider. May be "JDK" or "OPENSSL". Default is "JDK"
     provider: "JDK"
   }
-}
-
+},
 
+drill.metrics : {
+  context: "drillbit",
+  jmx: {
+    enabled : true
+  },
+  log: {
+    enabled : false,
+    interval : 60
+  }
+}
diff --git a/distribution/src/resources/sqlline b/distribution/src/resources/sqlline
index 1473ef0794..3f982bb681 100644
--- a/distribution/src/resources/sqlline
+++ b/distribution/src/resources/sqlline
@@ -76,6 +76,11 @@ if ! $is_cygwin; then
   DRILL_SHELL_OPTS="$DRILL_SHELL_OPTS --color=true"
 fi
 
+# To add nohup support for SQLline script
+if [[ ( ! $(ps -o stat= -p $$) =~ "+" ) && ! ( -p /dev/stdin ) ]]; then
+    export SQLLINE_JAVA_OPTS="$SQLLINE_JAVA_OPTS -Djline.terminal=jline.UnsupportedTerminal"
+fi
+
 SHELL_OPTS="$DRILL_SHELL_JAVA_OPTS $SQLLINE_JAVA_OPTS $DRILL_SHELL_LOG_OPTS $CLIENT_GC_OPTS"
 CMD="$JAVA $SHELL_OPTS -cp $CP sqlline.SqlLine -d org.apache.drill.jdbc.Driver --maxWidth=10000"
 
diff --git a/distribution/src/resources/storage-plugins-override-example.conf b/distribution/src/resources/storage-plugins-override-example.conf
new file mode 100644
index 0000000000..360ba2c0a4
--- /dev/null
+++ b/distribution/src/resources/storage-plugins-override-example.conf
@@ -0,0 +1,65 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# This file involves storage plugins configs, which can be updated on the Drill start-up.
+# This file is in HOCON format, see https://github.com/typesafehub/config/blob/master/HOCON.md for more information.
+
+  "storage":{
+    cp: {
+      type: "file",
+      connection: "classpath:///",
+      formats: {
+        "csv" : {
+          type: "text",
+          extensions: [ "csv" ],
+          delimiter: ","
+        }
+      }
+    }
+  }
+  "storage":{
+    dfs: {
+      type: "file",
+      connection: "hdfs:///",
+      workspaces: {
+        "root": {
+          "location": "/",
+          "writable": false,
+          "defaultInputFormat": null,
+          "allowAccessOutsideWorkspace": false
+        }
+      },
+      formats: {
+        "parquet": {
+          "type": "parquet"
+        }
+      },
+      enabled: false
+    }
+  }
+  "storage":{
+    mongo : {
+      type:"mongo",
+      connection:"mongodb://test_host:27017/",
+      enabled: true
+    }
+  }
+  "storage": {
+    openTSDB: {
+      type: "openTSDB",
+      connection: "http://localhost:8888",
+      enabled: true
+    }
+  }
diff --git a/docs/dev/DevDocs.md b/docs/dev/DevDocs.md
index 3fb76157d1..9b6ec09bb5 100644
--- a/docs/dev/DevDocs.md
+++ b/docs/dev/DevDocs.md
@@ -11,3 +11,7 @@ For information about how to do integration and unit testing in Drill see [Testi
 ## License Headers
 
 For more information about working with license headers see [LicenseHeaders.md](LicenseHeaders.md)
+
+## Javadocs
+
+For more info about generating and using javadocs see [Javadocs.md](Javadocs.md)
diff --git a/docs/dev/Javadocs.md b/docs/dev/Javadocs.md
new file mode 100644
index 0000000000..3a8365d90e
--- /dev/null
+++ b/docs/dev/Javadocs.md
@@ -0,0 +1,20 @@
+# Javadocs
+
+## Building
+
+ 1. Make sure you are in drill's root project directory.
+ 1. Build the project:
+    ```
+    mvn -T 1C clean install -DskipTests
+    ```
+ 1. Run:
+    ```
+    mvn javadoc:aggregate
+    ```
+ 1. The javadocs are generated and stored in `target/site/apidocs`.
+
+## Viewing In IntelliJ
+
+ 1. Go to `target/set/apidocs/index.html` in IntelliJ's project view.
+ 2. Right click on `target/set/apidocs/index.html`
+ 3. Select **Open in Browser**.
diff --git a/drill-yarn/pom.xml b/drill-yarn/pom.xml
index 7cb2a93463..a2238c6883 100644
--- a/drill-yarn/pom.xml
+++ b/drill-yarn/pom.xml
@@ -43,7 +43,7 @@
          </configuration>
       </plugin>
         <!-- See http://stackoverflow.com/questions/31173467/maven-cannot-find-git-dotgitdirectory -->
-  <!-- 
+  <!--
       <plugin>
         <groupId>pl.project13.maven</groupId>
         <artifactId>git-commit-id-plugin</artifactId>
@@ -65,7 +65,7 @@
      -->
     </plugins>
   </build>
-  
+
   <dependencies>
 
     <!-- For YARN client. -->
@@ -75,7 +75,7 @@
       <artifactId>hadoop-yarn-api</artifactId>
       <scope>compile</scope>
     </dependency>
-    
+
     <!--  Included as a reference because this seems to be the only
           way to pick up profile-specific jars. -->
 
@@ -96,17 +96,27 @@
       <artifactId>drill-protocol</artifactId>
       <version>${project.version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.apache.curator</groupId>
+      <artifactId>curator-client</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.curator</groupId>
+      <artifactId>curator-framework</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.curator</groupId>
+      <artifactId>curator-x-discovery</artifactId>
+    </dependency>
     <dependency>
       <groupId>org.apache.curator</groupId>
       <artifactId>curator-test</artifactId>
-      <scope>test</scope>
     </dependency>
 
     <!-- For configuration -->
     <dependency>
       <groupId>com.typesafe</groupId>
       <artifactId>config</artifactId>
-      <version>1.0.0</version>
     </dependency>
 
     <!-- Logging -->
@@ -116,7 +126,7 @@
       <version>1.0.13</version>
       <scope>test</scope>
     </dependency>
-    
+
     <!--  Testing -->
     <dependency>
       <groupId>org.apache.drill</groupId>
@@ -126,21 +136,4 @@
       <scope>test</scope>
     </dependency>
   </dependencies>
-
-  <dependencyManagement>
-   <dependencies>
-	   <dependency>
-	      <groupId>org.apache.curator</groupId>
-	      <artifactId>curator-test</artifactId>
-	      <version>2.7.1</version>
-	      <scope>test</scope>
-	      <exclusions>
-          <exclusion>
-            <groupId>log4j</groupId>
-            <artifactId>log4j</artifactId>
-          </exclusion>
-	      </exclusions>
-	    </dependency>
-    </dependencies>
-  </dependencyManagement>
 </project>
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/ClusterController.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/ClusterController.java
index 6aaa18b118..3fcc35f557 100644
--- a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/ClusterController.java
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/ClusterController.java
@@ -62,7 +62,6 @@
    *
    * @param containers
    *          the set of containers provided by YARN
-   * @return the set of tasks to launch
    */
 
   void containersAllocated(List<Container> containers);
@@ -146,8 +145,6 @@
    * Called by the main thread to wait for the normal shutdown of the
    * controller. Such shutdown occurs when the admin sends a sutdown
    * command from the UI or REST API.
-   *
-   * @return
    */
 
   boolean waitForCompletion();
@@ -176,8 +173,6 @@
   /**
    * Return the target number of tasks that the controller seeks to maintain.
    * This is the sum across all pools.
-   *
-   * @return
    */
 
   int getTargetCount();
@@ -190,17 +185,16 @@
    * when reducing cluster size.
    *
    * @param id
-   * @return
    */
 
   boolean cancelTask(int id);
 
   /**
    * Whether this distribution of YARN supports disk resources.
-   * @return
+   * @return True if this distribution of YARN supports disk resources. False otherwise.
    */
 
   boolean supportsDiskResource();
 
   int getFreeNodeCount();
-}
\ No newline at end of file
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/ClusterControllerImpl.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/ClusterControllerImpl.java
index 3c011ec54b..e611b6f025 100644
--- a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/ClusterControllerImpl.java
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/ClusterControllerImpl.java
@@ -311,7 +311,7 @@ private void adjustTasks(long curTime) {
    * This approximation <b>does not</b> consider whether the node
    * has sufficient resources to run a task; only whether the node
    * itself exists.
-   * @return
+   * @return The approximate number of free YARN nodes.
    */
 
   @Override
@@ -782,4 +782,4 @@ public boolean supportsDiskResource() {
 
   @Override
   public void registryDown() { shutDown( ); }
-}
\ No newline at end of file
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/PersistentTaskScheduler.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/PersistentTaskScheduler.java
index 73a045fa67..bfa636d49e 100644
--- a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/PersistentTaskScheduler.java
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/PersistentTaskScheduler.java
@@ -79,7 +79,7 @@ public void completed(Task task) { }
    * running tasks to target level. Thus, a persistent cluster
    * will normally report 100% progress.
    *
-   * @return
+   * @return The progress of persistent tasks.
    */
 
   @Override
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/Scheduler.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/Scheduler.java
index 7f8be0c9b4..bb1ce9dfd8 100644
--- a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/Scheduler.java
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/Scheduler.java
@@ -72,8 +72,6 @@
   /**
    * Whether tasks from this scheduler should incorporate app startup/shutdown
    * acknowledgements (acks) into the task lifecycle.
-   *
-   * @return
    */
 
   boolean isTracked();
@@ -83,7 +81,7 @@
   /**
    * Get the desired number of running tasks.
    *
-   * @return
+   * @return The desired number of running tasks
    */
   int getTarget();
 
@@ -119,7 +117,7 @@
    * Return an estimate of progress given as a ratio of (work completed, total
    * work).
    *
-   * @return
+   * @return Estimate of progress.
    */
   int[] getProgress();
 
@@ -135,7 +133,7 @@
   /**
    * For reporting, get the YARN resources requested by processes in
    * this pool.
-   * @return
+   * @return The request spec.
    */
 
   ContainerRequestSpec getResource( );
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/SchedulerState.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/SchedulerState.java
index 7a1f8bd027..39e81ce65e 100644
--- a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/SchedulerState.java
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/SchedulerState.java
@@ -29,7 +29,7 @@
   /**
    * The number of tasks in any active (non-ended) lifecycle state.
    *
-   * @return
+   * @return The number of tasks in any active (non-ended) lifecycle state.
    */
 
   int getTaskCount();
@@ -38,7 +38,8 @@
    * The number of active tasks that have been cancelled, but have not yet
    * ended.
    *
-   * @return
+   * @returnThe number of active tasks that have been cancelled, but have not yet
+   * ended.
    */
 
   int getCancelledTaskCount();
@@ -50,7 +51,9 @@
    * does not contain any tasks in this state which have previously been
    * cancelled.
    *
-   * @return
+   * @return The list of tasks awaiting a container request to be sent to YARN
+   * or for which a container request has been sent to YARN, but no container
+   * allocation has yet been received.
    */
 
   List<Task> getStartingTasks();
@@ -60,7 +63,7 @@
    * tasks are any task for which a container has been assigned, but has not yet
    * received a RM container completion event.
    *
-   * @return
+   * @return The list of active tasks that have not yet been cancelled.
    */
 
   List<Task> getActiveTasks();
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/SchedulerStateActions.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/SchedulerStateActions.java
index 65e8f2adc9..51884f734f 100644
--- a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/SchedulerStateActions.java
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/SchedulerStateActions.java
@@ -29,7 +29,7 @@
   /**
    * Returns the name of the scheduler associated with this task action group.
    *
-   * @return
+   * @return The name of the scheduler associated with this task action group.
    */
 
   String getName();
@@ -37,7 +37,7 @@
   /**
    * Returns the scheduler associated with this task group.
    *
-   * @return
+   * @return The scheduler associated with this task group.
    */
 
   Scheduler getScheduler();
@@ -80,7 +80,7 @@
    * differentiates the start state (when no tasks are active) from the end
    * state. The AM will not shut down until all task groups are done.
    *
-   * @return
+   * @return True if this task group is done. False otherwise.
    */
 
   boolean isDone();
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/AmRestApi.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/AmRestApi.java
index 21ddc4b036..0e6b405c25 100644
--- a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/AmRestApi.java
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/AmRestApi.java
@@ -134,9 +134,6 @@
    * set in the Drill-on-YARN configuration file. The purpose is simply to prevent
    * accidental cluster shutdown when experimenting with the REST API; this is
    * not meant to be a security mechanism.
-   *
-   * @param key
-   * @return
    */
 
   @Path("/stop")
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/AuthDynamicFeature.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/AuthDynamicFeature.java
index 12ee267be4..c882fe0273 100644
--- a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/AuthDynamicFeature.java
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/AuthDynamicFeature.java
@@ -44,7 +44,7 @@
  * login page.
  * <p>
  * Shameless copy of
- * {@link org.apache.drill.exec.server.rest.auth.DynamicFeature}; the two
+ * {@link org.apache.drill.exec.server.rest.auth.AuthDynamicFeature}; the two
  * implementations should be merged at some point. The difference is only the
  * log in/log out constant references.
  */
@@ -111,4 +111,4 @@ public void filter(ContainerRequestContext requestContext)
   public static boolean isUserLoggedIn(final SecurityContext sc) {
     return sc != null && sc.getUserPrincipal() != null;
   }
-}
\ No newline at end of file
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKClusterCoordinatorDriver.java b/drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKClusterCoordinatorDriver.java
index 3f83ff2bc3..171c4b0da1 100644
--- a/drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKClusterCoordinatorDriver.java
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKClusterCoordinatorDriver.java
@@ -99,7 +99,7 @@ public ZKClusterCoordinatorDriver() {
    * Specify connect string in the form: host:/zkRoot/clusterId
    *
    * @param connect
-   * @return
+   * @return This {@link ZKClusterCoordinatorDriver}.
    * @throws ZKConfigException
    */
   public ZKClusterCoordinatorDriver setConnect(String connect)
@@ -161,7 +161,7 @@ public ZKClusterCoordinatorDriver setPorts(int userPort, int controlPort,
    * Builds and starts the ZooKeeper cluster coordinator, translating any errors
    * that occur. After this call, the listener will start receiving messages.
    *
-   * @return
+   * @return This {@link ZKClusterCoordinatorDriver}.
    * @throws ZKRuntimeException
    *           if ZK startup fails
    */
@@ -201,7 +201,8 @@ public void removeDrillbitListener(DrillbitStatusListener listener) {
    * Returns the set of Drillbits registered at the time of the {@link #build()}
    * call. Should be empty for a cluster managed by YARN.
    *
-   * @return
+   * @return The set of Drillbits registered at the time of the {@link #build()}
+   * call.
    */
 
   public List<DrillbitEndpoint> getInitialEndpoints() {
@@ -213,7 +214,7 @@ public void removeDrillbitListener(DrillbitStatusListener listener) {
    * advertise the HTTP port, so it does not appear in the generated string.
    *
    * @param bit
-   * @return
+   * @return A string representation of a Drillbit.
    */
 
   public static String asString(DrillbitEndpoint bit) {
@@ -266,7 +267,7 @@ protected void stateChanged(ConnectionState newState) {
    * notification, and we received the disconnect notification log enough ago
    * that we assume that a timeout has occurred.
    *
-   * @return
+   * @return True if we think zookeeper has failed. False otherwise.
    */
 
   public boolean hasFailed() {
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKRegistry.java b/drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKRegistry.java
index 04265787c2..d05c5628b4 100644
--- a/drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKRegistry.java
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKRegistry.java
@@ -23,6 +23,7 @@
 import java.util.Map;
 import java.util.Set;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
@@ -134,7 +135,7 @@
     /**
      * ZK tracking state.
      *
-     * @see {@link State}
+     * See {@link org.apache.drill.yarn.zk.ZKRegistry.DrillbitTracker.State}
      */
 
     protected State state;
@@ -168,8 +169,6 @@ public DrillbitTracker(String key, Task task) {
      * that the task has come online. Tell the task to update its state to
      * record that the task is, in fact, registered in ZK. This indicates a
      * normal, healthy task.
-     *
-     * @param tracker
      */
 
     private void becomeRegistered() {
@@ -178,8 +177,6 @@ private void becomeRegistered() {
 
     /**
      * Mark that a YARN-managed Drillbit has dropped out of ZK.
-     *
-     * @param registryHandler
      */
 
     public void becomeUnregistered() {
@@ -492,7 +489,7 @@ private void taskCreated(Task task) {
    * cases where tasks hang in this state. This is a potential work-around.
    *
    * @param task
-   * @return
+   * @return True if the given task is regestered. False otherwise.
    */
 
   public synchronized boolean isRegistered(Task task) {
@@ -573,9 +570,9 @@ public void finish(RegistryHandler handler) {
    * Get the current registry for testing. Why for testing? Because this is
    * unsynchronized. In production code, the map may change out from under you.
    *
-   * @return
+   * @return The current registry.
    */
-
+  @VisibleForTesting
   protected Map<String, DrillbitTracker> getRegistryForTesting() {
     return registry;
   }
diff --git a/drill-yarn/src/test/java/org/apache/drill/yarn/scripts/ScriptUtils.java b/drill-yarn/src/test/java/org/apache/drill/yarn/scripts/ScriptUtils.java
index 3517cf8ee7..8a909a5d21 100644
--- a/drill-yarn/src/test/java/org/apache/drill/yarn/scripts/ScriptUtils.java
+++ b/drill-yarn/src/test/java/org/apache/drill/yarn/scripts/ScriptUtils.java
@@ -38,6 +38,7 @@
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.regex.Pattern;
@@ -66,7 +67,6 @@
     String args[] = {
       "-Dlog.path=/.*/drill/log/sqlline\\.log",
       "-Dlog.query.path=/.*/drill/log/sqlline_queries\\.json",
-      "-XX:MaxPermSize=512M",
       "sqlline\\.SqlLine",
       "-d",
       "org\\.apache\\.drill\\.jdbc\\.Driver",
@@ -123,11 +123,10 @@ public ScriptUtils fromDistrib(String distrib) {
       "-Xms4G",
       "-Xmx4G",
       "-XX:MaxDirectMemorySize=8G",
-      "-XX:MaxPermSize=512M",
       "-XX:ReservedCodeCacheSize=1G",
-      // Removed in Drill 1.8
-//      "-Ddrill\\.exec\\.enable-epoll=true",
-      "-XX:\\+CMSClassUnloadingEnabled",
+      "-Ddrill\\.exec\\.enable-epoll=false",
+      // Removed in Drill 1.14
+      //"-XX:\\+CMSClassUnloadingEnabled",
       "-XX:\\+UseG1GC",
       "org\\.apache\\.drill\\.exec\\.server\\.Drillbit",
       "-Dlog\\.path=/.*/script-test/drill/log/drillbit\\.log",
@@ -197,7 +196,8 @@ public ScriptUtils fromDistrib(String distrib) {
       "sqlline",
       //sqlline.bat
       //submit_plan
-      "yarn-drillbit.sh"
+      "yarn-drillbit.sh",
+      "auto-setup.sh"
   };
 
   /**
@@ -286,24 +286,89 @@ public void writeFile(File file, String contents) throws IOException {
     }
   }
 
+  public void writeEnvFile(PrintWriter out, String key, String value, boolean overrideValue) {
+    out.print("export ");
+    out.print(key);
+    out.print("=");
+
+    if (!overrideValue) {
+      out.print("${");
+      out.print(key);
+      out.print(":-");
+    }
+    out.print("\"");
+    out.print(value);
+    out.print("\"");
+
+    if (!overrideValue) {
+      out.print("}");
+    }
+    out.println();
+  }
+
   /**
    * Create a drill-env.sh or distrib-env.sh file with the given environment in
    * the recommended format.
+   * different formats based on overrideValue flag
+   *
+   * @param file - File instance to set environment variables in
+   * @param env - Environment to be placed inside File
+   * @param overrideValue - true - Set environment value such that it overrides previously set value
+   *                      - false - Set environment value in recommended format.
    */
-
-  public void createEnvFile(File file, Map<String, String> env)
+  public void createEnvFile(File file, Map<String, String> env, boolean overrideValue)
       throws IOException {
     try (PrintWriter out = new PrintWriter(new FileWriter(file))) {
       out.println("#!/usr/bin/env bash");
       for (String key : env.keySet()) {
         String value = env.get(key);
-        out.print("export ");
-        out.print(key);
-        out.print("=${");
-        out.print(key);
-        out.print(":-\"");
-        out.print(value);
-        out.println("\"}");
+        writeEnvFile(out, key, value, overrideValue);
+      }
+    }
+  }
+
+  /**
+   * Creates a drill-env.sh or distrib-env.sh file with the given environment under
+   * a given condition. If size of env map is smaller than condition map then last
+   * env entry is repeated for rest of conditions.
+   *
+   * @param file - File instance to set environment and condition in
+   * @param condition - Conditions to guard environment variable
+   * @param env - Environment to be placed inside File
+   * @param overrideValue - true - Set environment value such that it overrides previously set value
+   *                      - false - Set environment value in recommended format.
+   *
+   */
+  public void createEnvFileWithCondition(File file, Map<String, String> condition,
+                                         Map<String, String> env, boolean overrideValue) throws IOException {
+    if (env.size() == 0 || condition.size() == 0) {
+      return;
+    }
+
+    final Iterator envIterator = env.entrySet().iterator();
+    Map.Entry currentEnv = (Map.Entry) envIterator.next();
+
+    try (PrintWriter out = new PrintWriter(new FileWriter(file))) {
+      out.println("#!/usr/bin/env bash");
+
+      for (String condKey : condition.keySet()) {
+        String condValue = condition.get(condKey);
+        out.print("if [ \"$");
+        out.print(condKey);
+        out.print("\" = \"");
+        out.print(condValue);
+        out.println("\" ]; then");
+
+        final String envKey = currentEnv.getKey().toString();
+        final String envValue = currentEnv.getValue().toString();
+        writeEnvFile(out, envKey, envValue, overrideValue);
+
+        out.println("fi");
+        out.println();
+
+        if (envIterator.hasNext()) {
+          currentEnv = (Map.Entry) envIterator.next();
+        }
       }
     }
   }
@@ -342,7 +407,8 @@ private void buildFromDistrib() {
    * Consume the input from a stream, specifically the stderr or stdout stream
    * from a process.
    *
-   * @see http://stackoverflow.com/questions/14165517/processbuilder-forwarding-stdout-and-stderr-of-started-processes-without-blocki
+   * @link http://stackoverflow.com/questions/14165517/processbuilder-forwarding-stdout-and-stderr-of-started-processes
+   * -without-blocki
    */
 
   private static class StreamGobbler extends Thread {
diff --git a/drill-yarn/src/test/java/org/apache/drill/yarn/scripts/TestScripts.java b/drill-yarn/src/test/java/org/apache/drill/yarn/scripts/TestScripts.java
index 5f6b5bb005..38279f8a4a 100644
--- a/drill-yarn/src/test/java/org/apache/drill/yarn/scripts/TestScripts.java
+++ b/drill-yarn/src/test/java/org/apache/drill/yarn/scripts/TestScripts.java
@@ -159,14 +159,6 @@ public void testStockCombined() throws IOException {
       result.validateArgRegex("-Xloggc:.*/" + logTail);
     }
 
-    // Max Perm Size
-
-    {
-      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN)
-          .addEnv("DRILLBIT_MAX_PERM", "600M").run();
-      result.validateArg("-XX:MaxPermSize=600M");
-    }
-
     // Code cache size
 
     {
@@ -346,9 +338,8 @@ private void doEnvFileTest(String fileName) throws IOException {
     drillEnv.put("DRILL_HEAP", "5G");
     drillEnv.put("DRILL_MAX_DIRECT_MEMORY", "7G");
     drillEnv.put("SERVER_LOG_GC", "1");
-    drillEnv.put("DRILLBIT_MAX_PERM", "600M");
     drillEnv.put("DRILLBIT_CODE_CACHE_SIZE", "2G");
-    context.createEnvFile(new File(siteDir, fileName), drillEnv);
+    context.createEnvFile(new File(siteDir, fileName), drillEnv, false);
 
     {
       RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN).run();
@@ -358,8 +349,7 @@ private void doEnvFileTest(String fileName) throws IOException {
           propArg,
           "-Xms5G", "-Xmx5G",
           "-XX:MaxDirectMemorySize=7G",
-          "-XX:ReservedCodeCacheSize=2G",
-          "-XX:MaxPermSize=600M"
+          "-XX:ReservedCodeCacheSize=2G"
       };
 
       result.validateArgs(expectedArgs);
@@ -378,12 +368,50 @@ private void doEnvFileTest(String fileName) throws IOException {
           .run();
       assertEquals(0, result.returnCode);
       result.validateArg("-XX:MaxDirectMemorySize=9G");
-      result.validateArg("-XX:MaxPermSize=600M");
       String logTail = context.testDrillHome.getName() + "/log/drillbit.gc";
       assertFalse(result.containsArgRegex("-Xloggc:.*/" + logTail));
     }
   }
 
+  @Test
+  public void testDistribEnvWithNegativeCond() throws IOException {
+    // Construct condition map
+    final Map<String, String> conditions = new HashMap<>();
+    conditions.put("DRILLBIT_CONTEXT", "0");
+    final String expectedArgs[] = {"-XX:ReservedCodeCacheSize=1G"};
+    doEnvFileWithConditionTest("distrib-env.sh", conditions, expectedArgs);
+  }
+
+  @Test
+  public void testDistribEnvWithPositiveCond() throws IOException {
+    // Construct condition map
+    final Map<String, String> conditions = new HashMap<>();
+    conditions.put("DRILLBIT_CONTEXT", "1");
+    final String expectedArgs[] = {"-XX:ReservedCodeCacheSize=2G"};
+    doEnvFileWithConditionTest("distrib-env.sh", conditions, expectedArgs);
+  }
+
+  /**
+   * Implementation of the drill-env.sh or distrib-env.sh tests with conditions
+   * guarding environment variables.
+   */
+  private void doEnvFileWithConditionTest(String fileName, Map<String, String> conditions,
+                                          String[] expectedArgs) throws IOException {
+    context.createMockDistrib();
+    File siteDir = new File(context.testDrillHome, "conf");
+    context.createMockConf(siteDir);
+
+    // Set a property in the env file.
+    Map<String, String> drillEnv = new HashMap<>();
+    drillEnv.put("DRILLBIT_CODE_CACHE_SIZE", "2G");
+    context.createEnvFileWithCondition(new File(siteDir, fileName), conditions, drillEnv, false);
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN).run();
+      assertEquals(0, result.returnCode);
+      result.validateArgs(expectedArgs);
+    }
+  }
+
   /**
    * Test that drill-env.sh overrides distrib-env.sh, and that the environment
    * overrides both. Assumes the basics were tested above.
@@ -400,13 +428,12 @@ public void testDrillAndDistribEnv() throws IOException {
     Map<String, String> distribEnv = new HashMap<>();
     distribEnv.put("DRILL_HEAP", "5G");
     distribEnv.put("DRILL_MAX_DIRECT_MEMORY", "7G");
-    distribEnv.put("DRILLBIT_MAX_PERM", "600M");
-    context.createEnvFile(new File(siteDir, "distrib-env.sh"), distribEnv);
+    context.createEnvFile(new File(siteDir, "distrib-env.sh"), distribEnv, false);
 
     Map<String, String> drillEnv = new HashMap<>();
     drillEnv.put("DRILL_HEAP", "6G");
     drillEnv.put("DRILL_MAX_DIRECT_MEMORY", "9G");
-    context.createEnvFile(new File(siteDir, "drill-env.sh"), drillEnv);
+    context.createEnvFile(new File(siteDir, "drill-env.sh"), drillEnv, false);
 
     {
       RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN).run();
@@ -414,8 +441,7 @@ public void testDrillAndDistribEnv() throws IOException {
       String expectedArgs[] = {
           "-Xms6G", "-Xmx6G",
           "-XX:MaxDirectMemorySize=9G",
-          "-XX:MaxPermSize=600M",
-          "-XX:ReservedCodeCacheSize=1G" // Default
+          "-XX:ReservedCodeCacheSize=1024m" // Default
       };
 
       result.validateArgs(expectedArgs);
@@ -428,8 +454,7 @@ public void testDrillAndDistribEnv() throws IOException {
       String expectedArgs[] = {
           "-Xms6G", "-Xmx6G",
           "-XX:MaxDirectMemorySize=5G",
-          "-XX:MaxPermSize=600M",
-          "-XX:ReservedCodeCacheSize=1G" // Default
+          "-XX:ReservedCodeCacheSize=1024m" // Default
       };
 
       result.validateArgs(expectedArgs);
@@ -498,19 +523,17 @@ public void testSiteDir() throws IOException {
     Map<String, String> distribEnv = new HashMap<>();
     distribEnv.put("DRILL_HEAP", "5G");
     distribEnv.put("DRILL_MAX_DIRECT_MEMORY", "7G");
-    distribEnv.put("DRILLBIT_MAX_PERM", "600M");
-    context.createEnvFile(new File(confDir, "distrib-env.sh"), distribEnv);
+    context.createEnvFile(new File(confDir, "distrib-env.sh"), distribEnv, false);
 
     Map<String, String> drillEnv = new HashMap<>();
     drillEnv.put("DRILL_HEAP", "6G");
     drillEnv.put("DRILL_MAX_DIRECT_MEMORY", "9G");
-    context.createEnvFile(new File(siteDir, "drill-env.sh"), drillEnv);
+    context.createEnvFile(new File(siteDir, "drill-env.sh"), drillEnv, false);
 
     String expectedArgs[] = {
         "-Xms6G", "-Xmx6G",
         "-XX:MaxDirectMemorySize=9G",
-        "-XX:MaxPermSize=600M",
-        "-XX:ReservedCodeCacheSize=1G" // Default
+        "-XX:ReservedCodeCacheSize=1024m" // Default
     };
 
     // Site set using argument
@@ -611,8 +634,7 @@ public void testJavaLibDir() throws IOException {
 
     String prefix = "-Djava.library.path=";
     {
-      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN)
-          .run();
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN).run();
       assertFalse(result.containsArgRegex(prefix + ".*"));
       assertNull(result.libPath);
     }
@@ -874,7 +896,7 @@ public void testPidDir() throws IOException {
     File pidDir = context.createDir(new File(context.testDir, "pid"));
     Map<String, String> drillEnv = new HashMap<>();
     drillEnv.put("DRILL_PID_DIR", pidDir.getAbsolutePath());
-    context.createEnvFile(new File(siteDir, "drill-env.sh"), drillEnv);
+    context.createEnvFile(new File(siteDir, "drill-env.sh"), drillEnv, false);
 
     {
       RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_START)
@@ -905,7 +927,7 @@ public void testSiteDirWithDaemon() throws IOException {
 
     Map<String, String> drillEnv = new HashMap<>();
     drillEnv.put("DRILL_MAX_DIRECT_MEMORY", "9G");
-    context.createEnvFile(new File(siteDir, "drill-env.sh"), drillEnv);
+    context.createEnvFile(new File(siteDir, "drill-env.sh"), drillEnv, false);
 
     // Use the -site (--config) option.
 
@@ -948,7 +970,7 @@ public void testLogDirWithDaemon() throws IOException {
     context.removeDir(new File(context.testDrillHome, "log"));
     Map<String, String> drillEnv = new HashMap<>();
     drillEnv.put("DRILL_LOG_DIR", logsDir.getAbsolutePath());
-    context.createEnvFile(new File(siteDir, "drill-env.sh"), drillEnv);
+    context.createEnvFile(new File(siteDir, "drill-env.sh"), drillEnv, false);
 
     {
       DrillbitRun runner = new DrillbitRun(DrillbitRun.DRILLBIT_START);
@@ -1122,7 +1144,6 @@ public void testSqlline() throws IOException {
       drillEnv.put("DRILL_HEAP", "5G");
       drillEnv.put("DRILL_MAX_DIRECT_MEMORY", "7G");
       drillEnv.put("SERVER_LOG_GC", "1");
-      drillEnv.put("DRILLBIT_MAX_PERM", "600M");
       drillEnv.put("DRILLBIT_CODE_CACHE_SIZE", "2G");
       RunResult result = new ScriptRunner("sqlline")
           .withEnvironment(drillEnv)
@@ -1138,11 +1159,9 @@ public void testSqlline() throws IOException {
 
       Map<String, String> shellEnv = new HashMap<>();
       shellEnv.put("CLIENT_GC_OPTS", "-XX:+UseG1GC");
-      shellEnv.put("SQLLINE_JAVA_OPTS", "-XX:MaxPermSize=256M");
       RunResult result = new ScriptRunner("sqlline")
           .withEnvironment(shellEnv)
           .run();
-      assertTrue(result.containsArg("-XX:MaxPermSize=256M"));
       assertTrue(result.containsArg("-XX:+UseG1GC"));
     }
     {
@@ -1156,7 +1175,6 @@ public void testSqlline() throws IOException {
       drillEnv.put("DRILL_HEAP", "5G");
       drillEnv.put("DRILL_MAX_DIRECT_MEMORY", "7G");
       drillEnv.put("SERVER_LOG_GC", "1");
-      drillEnv.put("DRILLBIT_MAX_PERM", "600M");
       drillEnv.put("DRILLBIT_CODE_CACHE_SIZE", "2G");
       drillEnv.put("DRILL_EMBEDDED", "1");
       RunResult result = new ScriptRunner("sqlline")
@@ -1168,7 +1186,6 @@ public void testSqlline() throws IOException {
           "-Xms5G", "-Xmx5G",
           "-XX:MaxDirectMemorySize=7G",
           "-XX:ReservedCodeCacheSize=2G",
-          "-XX:MaxPermSize=600M"
       };
 
       result.validateArgs(expectedArgs);
@@ -1176,6 +1193,80 @@ public void testSqlline() throws IOException {
     }
   }
 
+  /**
+   * Test to verify no effect of DRILLBIT_CONTEXT for Sqlline.
+   * @throws IOException
+   */
+  @Test
+  public void testSqllineWithDrillbitContextEnv() throws IOException {
+    context.createMockDistrib();
+    File siteDir = new File(context.testDrillHome, "conf");
+    context.createMockConf(siteDir);
+
+    // Test when SQLLINE_JAVA_OPTS is overriden inside a condition for
+    // DRILLBIT_CONTEXT = 0, then there is no effect
+    {
+      // Create a condition variable to be placed in distrib-env.sh
+      Map<String, String> conditions = new HashMap<>();
+      conditions.put("DRILLBIT_CONTEXT", "0");
+
+      // Create environment variable to be placed inside a condition in distrib-env.sh
+      Map<String, String> drillEnv = new HashMap<>();
+      drillEnv.put("SQLLINE_JAVA_OPTS", "-XX:MaxPermSize=256M");
+
+      // Create the environment variable file overriding SQLLINE_JAVA_OPTS
+      context.createEnvFileWithCondition(new File(siteDir, "distrib-env.sh"), conditions, drillEnv, true);
+
+      // Expected value of the property
+      String expectedArgs[] = {"-XX:MaxPermSize=256M"};
+
+      // Run the test and match the output with expectedArgs
+      RunResult result = new ScriptRunner("sqlline").run();
+      assertEquals(0, result.returnCode);
+      result.validateJava();
+      result.validateClassPath(ScriptUtils.stdCp);
+      // Since by default MaxPermSize is not set anymore for Sqlline. It's removed in 1.13
+      assertFalse(result.containsArgsRegex(expectedArgs));
+    }
+
+    // Test when SQLLINE_JAVA_OPTS is overriden inside a condition for
+    // DRILLBIT_CONTEXT = 1, then there is no effect
+    {
+      Map<String, String> conditions = new HashMap<>();
+      conditions.put("DRILLBIT_CONTEXT", "1");
+
+      Map<String, String> drillEnv = new HashMap<>();
+      drillEnv.put("SQLLINE_JAVA_OPTS", "-XX:MaxPermSize=256M");
+      String expectedArgs[] = {"-XX:MaxPermSize=256M"};
+
+      // Create the environment variable file overriding SQLLINE_JAVA_OPTS
+      context.createEnvFileWithCondition(new File(siteDir, "distrib-env.sh"), conditions, drillEnv, true);
+      RunResult result = new ScriptRunner("sqlline").run();
+      assertEquals(0, result.returnCode);
+      result.validateJava();
+      result.validateClassPath(ScriptUtils.stdCp);
+      // Since by default MaxPermSize is not set anymore for Sqlline. It's removed in 1.13
+      assertFalse(result.containsArgsRegex(expectedArgs));
+    }
+
+    // Test when SQLLINE_JAVA_OPTS is overriden without condition for
+    // DRILLBIT_CONTEXT then the environment variable is updated
+    {
+      Map<String, String> drillEnv = new HashMap<>();
+      drillEnv.put("SQLLINE_JAVA_OPTS", "-XX:MaxPermSize=256M");
+
+      // Create the environment variable file overriding SQLLINE_JAVA_OPTS without any condition
+      // around it.
+      String expectedArgs[] = {"-XX:MaxPermSize=256M"};
+      context.createEnvFile(new File(siteDir, "distrib-env.sh"), drillEnv, true);
+      RunResult result = new ScriptRunner("sqlline").run();
+      assertEquals(0, result.returnCode);
+      result.validateJava();
+      result.validateClassPath(ScriptUtils.stdCp);
+      assertTrue(result.containsArgsRegex(expectedArgs));
+    }
+  }
+
   /**
    * Verify that the sqlline client works with the --site option by customizing
    * items in the site directory.
diff --git a/exec/java-exec/pom.xml b/exec/java-exec/pom.xml
index 345e240143..3e1a118097 100644
--- a/exec/java-exec/pom.xml
+++ b/exec/java-exec/pom.xml
@@ -97,7 +97,11 @@
       <artifactId>univocity-parsers</artifactId>
       <version>1.3.0</version>
     </dependency>
-
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-text</artifactId>
+      <version>1.4</version>
+    </dependency>
     <dependency>
       <groupId>org.apache.commons</groupId>
       <artifactId>commons-math</artifactId>
@@ -205,6 +209,10 @@
       <artifactId>jackson-module-afterburner</artifactId>
       <version>${jackson.version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.honton.chas.hocon</groupId>
+      <artifactId>jackson-dataformat-hocon</artifactId>
+    </dependency>
     <dependency>
       <groupId>org.glassfish.jersey.ext</groupId>
       <artifactId>jersey-mvc-freemarker</artifactId>
@@ -249,92 +257,17 @@
     <dependency>
       <groupId>org.apache.parquet</groupId>
       <artifactId>parquet-hadoop</artifactId>
-      <version>${parquet.version}</version>
       <exclusions>
-        <exclusion>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-client</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-common</artifactId>
-        </exclusion>
       </exclusions>
     </dependency>
     <dependency>
       <groupId>org.apache.parquet</groupId>
       <artifactId>parquet-format</artifactId>
-      <version>2.3.0-incubating</version>
-      <exclusions>
-        <exclusion>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-client</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-common</artifactId>
-        </exclusion>
-      </exclusions>
     </dependency>
     <dependency>
       <groupId>org.apache.parquet</groupId>
       <artifactId>parquet-common</artifactId>
       <version>${parquet.version}</version>
-      <exclusions>
-        <exclusion>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-client</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-common</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.parquet</groupId>
-      <artifactId>parquet-jackson</artifactId>
-      <version>${parquet.version}</version>
-      <exclusions>
-        <exclusion>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-client</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-common</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.parquet</groupId>
-      <artifactId>parquet-encoding</artifactId>
-      <version>${parquet.version}</version>
-      <exclusions>
-        <exclusion>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-client</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-common</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.parquet</groupId>
-      <artifactId>parquet-generator</artifactId>
-      <version>${parquet.version}</version>
-      <exclusions>
-        <exclusion>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-client</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-common</artifactId>
-        </exclusion>
-      </exclusions>
     </dependency>
     <dependency>
       <groupId>javax.inject</groupId>
@@ -381,40 +314,25 @@
       <artifactId>jcommander</artifactId>
       <version>1.30</version>
     </dependency>
+    <dependency>
+      <groupId>org.apache.curator</groupId>
+      <artifactId>curator-client</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.curator</groupId>
+      <artifactId>curator-framework</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.curator</groupId>
+      <artifactId>curator-recipes</artifactId>
+    </dependency>
     <dependency>
       <groupId>org.apache.curator</groupId>
       <artifactId>curator-x-discovery</artifactId>
-      <version>2.7.1</version>
-      <exclusions>
-        <exclusion>
-          <artifactId>slf4j-log4j12</artifactId>
-          <groupId>org.slf4j</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>log4j</artifactId>
-          <groupId>log4j</groupId>
-        </exclusion>
-        <exclusion>
-            <groupId>io.netty</groupId>
-            <artifactId>netty</artifactId>
-        </exclusion>
-        <exclusion>
-            <groupId>io.netty</groupId>
-            <artifactId>netty-all</artifactId>
-        </exclusion>
-      </exclusions>
     </dependency>
     <dependency>
       <groupId>org.apache.curator</groupId>
       <artifactId>curator-test</artifactId>
-      <version>2.7.1</version>
-      <scope>test</scope>
-      <exclusions>
-        <exclusion>
-            <groupId>io.netty</groupId>
-            <artifactId>netty</artifactId>
-        </exclusion>
-      </exclusions>
     </dependency>
     <dependency>
       <groupId>com.carrotsearch</groupId>
@@ -587,7 +505,7 @@
     <dependency>
       <groupId>org.apache.maven</groupId>
       <artifactId>maven-embedder</artifactId>
-      <version>3.5.3</version>
+      <version>${maven.embedder.version}</version>
       <scope>test</scope>
     </dependency>
     <dependency>
@@ -632,6 +550,11 @@
         </exclusion>
       </exclusions>
     </dependency>
+    <dependency>
+      <groupId>com.drewnoakes</groupId>
+      <artifactId>metadata-extractor</artifactId>
+      <version>2.11.0</version>
+    </dependency>
   </dependencies>
 
   <profiles>
@@ -719,6 +642,11 @@
 
   <build>
     <plugins>
+      <plugin>
+        <!-- Creating a test artifact because javadoc needs to be able to find test classes -->
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+      </plugin>
       <plugin>
         <artifactId>maven-resources-plugin</artifactId>
         <executions>
diff --git a/exec/java-exec/src/main/codegen/config.fmpp b/exec/java-exec/src/main/codegen/config.fmpp
index 50f110dd18..e23397445c 100644
--- a/exec/java-exec/src/main/codegen/config.fmpp
+++ b/exec/java-exec/src/main/codegen/config.fmpp
@@ -43,6 +43,7 @@ data: {
     intervalNumericTypes:     tdd(../data/IntervalNumericTypes.tdd),
     extract:                  tdd(../data/ExtractTypes.tdd),
     sumzero:                  tdd(../data/SumZero.tdd),
+    singleValue:              tdd(../data/SingleValue.tdd),
     numericTypes:             tdd(../data/NumericTypes.tdd),
     casthigh:                 tdd(../data/CastHigh.tdd),
     countAggrTypes:           tdd(../data/CountAggrTypes.tdd)
diff --git a/exec/java-exec/src/main/codegen/data/AggrTypes1.tdd b/exec/java-exec/src/main/codegen/data/AggrTypes1.tdd
index 202f539cb1..3fb2601418 100644
--- a/exec/java-exec/src/main/codegen/data/AggrTypes1.tdd
+++ b/exec/java-exec/src/main/codegen/data/AggrTypes1.tdd
@@ -88,6 +88,52 @@
       {inputType: "Interval", outputType: "NullableInterval", runningType: "Interval", major: "Date", initialValue: "0"},
       {inputType: "NullableInterval", outputType: "NullableInterval", runningType: "Interval", major: "Date", initialValue: "0"}
      ]
+   },
+   {className: "AnyValue", funcName: "any_value", types: [
+       {inputType: "Bit", outputType: "NullableBit", runningType: "Bit", major: "Numeric"},
+       {inputType: "Int", outputType: "NullableInt", runningType: "Int", major: "Numeric"},
+       {inputType: "BigInt", outputType: "NullableBigInt", runningType: "BigInt", major: "Numeric"},
+       {inputType: "NullableBit", outputType: "NullableBit", runningType: "Bit", major: "Numeric"},
+       {inputType: "NullableInt", outputType: "NullableInt", runningType: "Int", major: "Numeric"},
+       {inputType: "NullableBigInt", outputType: "NullableBigInt", runningType: "BigInt", major: "Numeric"},
+       {inputType: "Float4", outputType: "NullableFloat4", runningType: "Float4", major: "Numeric"},
+       {inputType: "Float8", outputType: "NullableFloat8", runningType: "Float8", major: "Numeric"},
+       {inputType: "NullableFloat4", outputType: "NullableFloat4", runningType: "Float4", major: "Numeric"},
+       {inputType: "NullableFloat8", outputType: "NullableFloat8", runningType: "Float8", major: "Numeric"},
+       {inputType: "Date", outputType: "NullableDate", runningType: "Date", major: "Date", initialValue: "0"},
+       {inputType: "NullableDate", outputType: "NullableDate", runningType: "Date", major: "Date", initialValue: "0"},
+       {inputType: "TimeStamp", outputType: "NullableTimeStamp", runningType: "TimeStamp", major: "Date", initialValue: "0"},
+       {inputType: "NullableTimeStamp", outputType: "NullableTimeStamp", runningType: "TimeStamp", major: "Date", initialValue: "0"},
+       {inputType: "Time", outputType: "NullableTime", runningType: "Time", major: "Date", initialValue: "0"},
+       {inputType: "NullableTime", outputType: "NullableTime", runningType: "Time", major: "Date", initialValue: "0"},
+       {inputType: "IntervalDay", outputType: "NullableIntervalDay", runningType: "IntervalDay", major: "Date", initialValue: "0"},
+       {inputType: "NullableIntervalDay", outputType: "NullableIntervalDay", runningType: "IntervalDay", major: "Date", initialValue: "0"},
+       {inputType: "IntervalYear", outputType: "NullableIntervalYear", runningType: "IntervalYear", major: "Date", initialValue: "0"},
+       {inputType: "NullableIntervalYear", outputType: "NullableIntervalYear", runningType: "IntervalYear", major: "Date", initialValue: "0"},
+       {inputType: "Interval", outputType: "NullableInterval", runningType: "Interval", major: "Date", initialValue: "0"},
+       {inputType: "NullableInterval", outputType: "NullableInterval", runningType: "Interval", major: "Date", initialValue: "0"},
+       {inputType: "VarChar", outputType: "NullableVarChar", runningType: "VarChar", major: "VarBytes", initialValue: ""},
+       {inputType: "NullableVarChar", outputType: "NullableVarChar", runningType: "VarChar", major: "VarBytes", initialValue: ""},
+       {inputType: "VarBinary", outputType: "NullableVarBinary", runningType: "VarBinary", major: "VarBytes"},
+       {inputType: "NullableVarBinary", outputType: "NullableVarBinary", runningType: "VarBinary", major: "VarBytes"}
+       {inputType: "List", outputType: "List", runningType: "List", major: "Complex"}
+       {inputType: "Map", outputType: "Map", runningType: "Map", major: "Complex"}
+       {inputType: "RepeatedBit", outputType: "RepeatedBit", runningType: "RepeatedBit", major: "Complex"},
+       {inputType: "RepeatedInt", outputType: "RepeatedInt", runningType: "RepeatedInt", major: "Complex"},
+       {inputType: "RepeatedBigInt", outputType: "RepeatedBigInt", runningType: "RepeatedBigInt", major: "Complex"},
+       {inputType: "RepeatedFloat4", outputType: "RepeatedFloat4", runningType: "RepeatedFloat4", major: "Complex"},
+       {inputType: "RepeatedFloat8", outputType: "RepeatedFloat8", runningType: "RepeatedFloat8", major: "Complex"},
+       {inputType: "RepeatedDate", outputType: "RepeatedDate", runningType: "RepeatedDate", major: "Complex"},
+       {inputType: "RepeatedTimeStamp", outputType: "RepeatedTimeStamp", runningType: "RepeatedTimeStamp", major: "Complex"},
+       {inputType: "RepeatedTime", outputType: "RepeatedTime", runningType: "RepeatedTime", major: "Complex"},
+       {inputType: "RepeatedIntervalDay", outputType: "RepeatedIntervalDay", runningType: "RepeatedIntervalDay", major: "Complex"},
+       {inputType: "RepeatedIntervalYear", outputType: "RepeatedIntervalYear", runningType: "RepeatedIntervalYear", major: "Complex"},
+       {inputType: "RepeatedInterval", outputType: "RepeatedInterval", runningType: "RepeatedInterval", major: "Complex"},
+       {inputType: "RepeatedVarChar", outputType: "RepeatedVarChar", runningType: "RepeatedVarChar", major: "Complex"},
+       {inputType: "RepeatedVarBinary", outputType: "RepeatedVarBinary", runningType: "RepeatedVarBinary", major: "Complex"},
+       {inputType: "RepeatedList", outputType: "RepeatedList", runningType: "RepeatedList", major: "Complex"},
+       {inputType: "RepeatedMap", outputType: "RepeatedMap", runningType: "RepeatedMap", major: "Complex"}
+     ]
    }
   ]
 }
diff --git a/exec/java-exec/src/main/codegen/data/DecimalAggrTypes1.tdd b/exec/java-exec/src/main/codegen/data/DecimalAggrTypes1.tdd
index 7da2d071f1..003bbfa1b2 100644
--- a/exec/java-exec/src/main/codegen/data/DecimalAggrTypes1.tdd
+++ b/exec/java-exec/src/main/codegen/data/DecimalAggrTypes1.tdd
@@ -35,6 +35,12 @@
        {inputType: "VarDecimal", outputType: "NullableVarDecimal"},
        {inputType: "NullableVarDecimal", outputType: "NullableVarDecimal"}
       ]
+   },
+   {className: "AnyValue", funcName: "any_value", types: [
+       {inputType: "VarDecimal", outputType: "NullableVarDecimal"},
+       {inputType: "NullableVarDecimal", outputType: "NullableVarDecimal"}
+       {inputType: "RepeatedVarDecimal", outputType: "RepeatedVarDecimal"}
+      ]
    }
   ]
 }
diff --git a/exec/java-exec/src/main/codegen/data/SingleValue.tdd b/exec/java-exec/src/main/codegen/data/SingleValue.tdd
new file mode 100644
index 0000000000..a42fe3b456
--- /dev/null
+++ b/exec/java-exec/src/main/codegen/data/SingleValue.tdd
@@ -0,0 +1,62 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+# http:# www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+{
+types: [
+    {inputType: "Bit", outputType: "NullableBit", runningType: "Bit", major: "primitive"},
+    {inputType: "TinyInt", outputType: "NullableTinyInt", runningType: "TinyInt", major: "primitive"},
+    {inputType: "NullableTinyInt", outputType: "NullableTinyInt", runningType: "TinyInt", major: "primitive"},
+    {inputType: "UInt1", outputType: "NullableUInt1", runningType: "UInt1", major: "primitive"},
+    {inputType: "NullableUInt1", outputType: "NullableUInt1", runningType: "UInt1", major: "primitive"},
+    {inputType: "UInt2", outputType: "NullableUInt2", runningType: "UInt2", major: "primitive"},
+    {inputType: "NullableUInt2", outputType: "NullableUInt2", runningType: "UInt2", major: "primitive"},
+    {inputType: "SmallInt", outputType: "NullableSmallInt", runningType: "SmallInt", major: "primitive"},
+    {inputType: "NullableSmallInt", outputType: "NullableSmallInt", runningType: "SmallInt", major: "primitive"},
+    {inputType: "UInt4", outputType: "NullableUInt4", runningType: "UInt4", major: "primitive"},
+    {inputType: "NullableUInt4", outputType: "NullableUInt4", runningType: "UInt4", major: "primitive"},
+    {inputType: "UInt8", outputType: "NullableUInt8", runningType: "UInt8", major: "primitive"},
+    {inputType: "NullableUInt8", outputType: "NullableUInt8", runningType: "UInt8", major: "primitive"},
+    {inputType: "Int", outputType: "NullableInt", runningType: "Int", major: "primitive"},
+    {inputType: "BigInt", outputType: "NullableBigInt", runningType: "BigInt", major: "primitive"},
+    {inputType: "NullableBit", outputType: "NullableBit", runningType: "Bit", major: "primitive"},
+    {inputType: "NullableInt", outputType: "NullableInt", runningType: "Int", major: "primitive"},
+    {inputType: "NullableBigInt", outputType: "NullableBigInt", runningType: "BigInt", major: "primitive"},
+    {inputType: "Float4", outputType: "NullableFloat4", runningType: "Float4", major: "primitive"},
+    {inputType: "Float8", outputType: "NullableFloat8", runningType: "Float8", major: "primitive"},
+    {inputType: "NullableFloat4", outputType: "NullableFloat4", runningType: "Float4", major: "primitive"},
+    {inputType: "NullableFloat8", outputType: "NullableFloat8", runningType: "Float8", major: "primitive"},
+    {inputType: "Date", outputType: "NullableDate", runningType: "Date", major: "primitive"},
+    {inputType: "NullableDate", outputType: "NullableDate", runningType: "Date", major: "primitive"},
+    {inputType: "TimeStamp", outputType: "NullableTimeStamp", runningType: "TimeStamp", major: "primitive"},
+    {inputType: "NullableTimeStamp", outputType: "NullableTimeStamp", runningType: "TimeStamp", major: "primitive"},
+    {inputType: "Time", outputType: "NullableTime", runningType: "Time", major: "primitive"},
+    {inputType: "NullableTime", outputType: "NullableTime", runningType: "Time", major: "primitive"},
+    {inputType: "IntervalDay", outputType: "NullableIntervalDay", runningType: "IntervalDay", major: "IntervalDay"},
+    {inputType: "NullableIntervalDay", outputType: "NullableIntervalDay", runningType: "IntervalDay", major: "IntervalDay"},
+    {inputType: "IntervalYear", outputType: "NullableIntervalYear", runningType: "IntervalYear", major: "primitive"},
+    {inputType: "NullableIntervalYear", outputType: "NullableIntervalYear", runningType: "IntervalYear", major: "primitive"},
+    {inputType: "Interval", outputType: "NullableInterval", runningType: "Interval", major: "Interval"},
+    {inputType: "NullableInterval", outputType: "NullableInterval", runningType: "Interval", major: "Interval"},
+    {inputType: "VarDecimal", outputType: "NullableVarDecimal", runningType: "VarDecimal", major: "VarDecimal"},
+    {inputType: "NullableVarDecimal", outputType: "NullableVarDecimal", runningType: "VarDecimal", major: "VarDecimal"},
+    {inputType: "VarChar", outputType: "NullableVarChar", runningType: "VarChar", major: "bytes"},
+    {inputType: "NullableVarChar", outputType: "NullableVarChar", runningType: "VarChar", major: "bytes"},
+    {inputType: "Var16Char", outputType: "NullableVar16Char", runningType: "Var16Char", major: "bytes"},
+    {inputType: "NullableVar16Char", outputType: "NullableVar16Char", runningType: "Var16Char", major: "bytes"},
+    {inputType: "VarBinary", outputType: "NullableVarBinary", runningType: "VarBinary", major: "bytes"},
+    {inputType: "NullableVarBinary", outputType: "NullableVarBinary", runningType: "VarBinary", major: "bytes"}
+   ]
+}
diff --git a/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions1.java b/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions1.java
index ebf20e53e1..59d37157f7 100644
--- a/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions1.java
+++ b/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions1.java
@@ -61,11 +61,11 @@ public void setup() {
 	  value = new ${type.runningType}Holder();
 	  nonNullCount = new BigIntHolder();
 	  nonNullCount.value = 0;
-	<#if aggrtype.funcName == "sum">
+	<#if aggrtype.funcName == "sum" || aggrtype.funcName == "any_value">
 	  value.value = 0;
 	<#elseif aggrtype.funcName == "min">
     <#if type.runningType?starts_with("Bit")>
-        value.value = 1;
+      value.value = 1;
 	  <#elseif type.runningType?starts_with("Int")>
 	    value.value = Integer.MAX_VALUE;
 	  <#elseif type.runningType?starts_with("BigInt")>
@@ -77,7 +77,7 @@ public void setup() {
 	  </#if>
 	<#elseif aggrtype.funcName == "max">
     <#if type.runningType?starts_with("Bit")>
-        value.value = 0;
+      value.value = 0;
 	  <#elseif type.runningType?starts_with("Int")>
 	    value.value = Integer.MIN_VALUE;
 	  <#elseif type.runningType?starts_with("BigInt")>
@@ -110,19 +110,21 @@ public void add() {
 		  value.value = Float.isNaN(value.value) ? in.value : Math.min(value.value, in.value);
 		}
 	    <#elseif type.inputType?contains("Float8")>
-	    if(!Double.isNaN(in.value)) {
-	      value.value = Double.isNaN(value.value) ? in.value : Math.min(value.value, in.value);
-	    }
-        <#else>
+	  if(!Double.isNaN(in.value)) {
+	    value.value = Double.isNaN(value.value) ? in.value : Math.min(value.value, in.value);
+	  }
+      <#else>
 		value.value = Math.min(value.value, in.value);
-		</#if>
+		  </#if>
 	  <#elseif aggrtype.funcName == "max">
 	    value.value = Math.max(value.value,  in.value);
 	  <#elseif aggrtype.funcName == "sum">
 	    value.value += in.value;
 	  <#elseif aggrtype.funcName == "count">
 	    value.value++;
-	  <#else>
+    <#elseif aggrtype.funcName == "any_value">
+		  value.value = in.value;
+    <#else>
 	  // TODO: throw an error ? 
 	  </#if>
 	<#if type.inputType?starts_with("Nullable")>
@@ -143,7 +145,7 @@ public void output() {
   @Override
   public void reset() {
     nonNullCount.value = 0;
-	<#if aggrtype.funcName == "sum" || aggrtype.funcName == "count">
+	<#if aggrtype.funcName == "sum" || aggrtype.funcName == "count" || aggrtype.funcName == "any_value">
 	  value.value = 0;
 	<#elseif aggrtype.funcName == "min">
 	  <#if type.runningType?starts_with("Int")>
diff --git a/exec/java-exec/src/main/codegen/templates/ComplexAggrFunctions1.java b/exec/java-exec/src/main/codegen/templates/ComplexAggrFunctions1.java
new file mode 100644
index 0000000000..6aa92e3c09
--- /dev/null
+++ b/exec/java-exec/src/main/codegen/templates/ComplexAggrFunctions1.java
@@ -0,0 +1,120 @@
+/*
+ * 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.
+ */
+<@pp.dropOutputFile />
+
+
+
+<#list aggrtypes1.aggrtypes as aggrtype>
+<@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/gaggr/${aggrtype.className}ComplexFunctions.java" />
+
+<#include "/@includes/license.ftl" />
+
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
+
+<#-- A utility class that is used to generate java code for aggr functions that maintain a single -->
+<#-- running counter to hold the result.  This includes: ANY_VALUE. -->
+
+package org.apache.drill.exec.expr.fn.impl.gaggr;
+
+import org.apache.drill.exec.expr.DrillAggFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.expr.holders.*;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.apache.drill.exec.vector.complex.MapUtility;
+import org.apache.drill.exec.vector.complex.writer.*;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter.*;
+
+@SuppressWarnings("unused")
+
+public class ${aggrtype.className}ComplexFunctions {
+static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(${aggrtype.className}ComplexFunctions.class);
+
+<#list aggrtype.types as type>
+<#if type.major == "Complex">
+
+@FunctionTemplate(name = "${aggrtype.funcName}", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+public static class ${type.inputType}${aggrtype.className} implements DrillAggFunc{
+  @Param ${type.inputType}Holder inHolder;
+  @Workspace BigIntHolder nonNullCount;
+  @Output ComplexWriter writer;
+
+  public void setup() {
+    nonNullCount = new BigIntHolder();
+    nonNullCount.value = 0;
+  }
+
+  @Override
+  public void add() {
+  <#if type.inputType?starts_with("Nullable")>
+    sout: {
+    if (inHolder.isSet == 0) {
+    // processing nullable input and the value is null, so don't do anything...
+    break sout;
+    }
+  </#if>
+  <#if aggrtype.funcName == "any_value">
+    <#if type.runningType?starts_with("Map")>
+    if (nonNullCount.value == 0) {
+      org.apache.drill.exec.expr.fn.impl.MappifyUtility.createMap(inHolder.reader, writer, "any_value");
+    }
+    <#elseif type.runningType?starts_with("RepeatedMap")>
+    if (nonNullCount.value == 0) {
+      org.apache.drill.exec.expr.fn.impl.MappifyUtility.createRepeatedMapOrList(inHolder.reader, writer, "any_value");
+    }
+    <#elseif type.runningType?starts_with("List")>
+    if (nonNullCount.value == 0) {
+      org.apache.drill.exec.expr.fn.impl.MappifyUtility.createList(inHolder.reader, writer, "any_value");
+    }
+    <#elseif type.runningType?starts_with("RepeatedList")>
+    if (nonNullCount.value == 0) {
+      org.apache.drill.exec.expr.fn.impl.MappifyUtility.createRepeatedMapOrList(inHolder.reader, writer, "any_value");
+    }
+    <#elseif type.runningType?starts_with("Repeated")>
+    if (nonNullCount.value == 0) {
+      org.apache.drill.exec.expr.fn.impl.MappifyUtility.createList(inHolder.reader, writer, "any_value");
+    }
+    </#if>
+  </#if>
+    nonNullCount.value = 1;
+  <#if type.inputType?starts_with("Nullable")>
+    } // end of sout block
+  </#if>
+  }
+
+  @Override
+  public void output() {
+    //Do nothing since the complex writer takes care of everything!
+  }
+
+  @Override
+  public void reset() {
+  <#if aggrtype.funcName == "any_value">
+    nonNullCount.value = 0;
+  </#if>
+  }
+}
+</#if>
+</#list>
+}
+</#list>
\ No newline at end of file
diff --git a/exec/java-exec/src/main/codegen/templates/DateIntervalAggrFunctions1.java b/exec/java-exec/src/main/codegen/templates/DateIntervalAggrFunctions1.java
index f526575b02..8080ea76b5 100644
--- a/exec/java-exec/src/main/codegen/templates/DateIntervalAggrFunctions1.java
+++ b/exec/java-exec/src/main/codegen/templates/DateIntervalAggrFunctions1.java
@@ -131,7 +131,16 @@ public void add() {
     </#if>
 	  <#elseif aggrtype.funcName == "count">
 	    value.value++;
-	  <#else>
+    <#elseif aggrtype.funcName == "any_value">
+      <#if type.outputType?ends_with("Interval")>
+        value.days = in.days;
+        value.months = in.months;
+        value.milliseconds = in.milliseconds;
+      <#elseif type.outputType?ends_with("IntervalDay")>
+        value.days = in.days;
+        value.milliseconds = in.milliseconds;
+      </#if>
+    <#else>
 	  // TODO: throw an error ?
 	  </#if>
 	<#if type.inputType?starts_with("Nullable")>
diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/DecimalAggrTypeFunctions1.java b/exec/java-exec/src/main/codegen/templates/Decimal/DecimalAggrTypeFunctions1.java
index 6b23f92edc..7f4ca154cd 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/DecimalAggrTypeFunctions1.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/DecimalAggrTypeFunctions1.java
@@ -39,6 +39,8 @@
 import org.apache.drill.exec.expr.annotations.Output;
 import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.vector.complex.writer.*;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter.*;
 import javax.inject.Inject;
 import io.netty.buffer.DrillBuf;
 import org.apache.drill.exec.expr.holders.*;
@@ -124,6 +126,101 @@ public void reset() {
       nonNullCount.value = 0;
     }
   }
+  <#elseif aggrtype.funcName.contains("any_value") && type.inputType?starts_with("Repeated")>
+  @FunctionTemplate(name = "${aggrtype.funcName}",
+                    scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE,
+                    returnType = FunctionTemplate.ReturnType.DECIMAL_AGGREGATE)
+  public static class ${type.inputType}${aggrtype.className} implements DrillAggFunc {
+    @Param ${type.inputType}Holder in;
+    @Output ComplexWriter writer;
+    @Workspace BigIntHolder nonNullCount;
+
+    public void setup() {
+      nonNullCount = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      if (nonNullCount.value == 0) {
+        org.apache.drill.exec.expr.fn.impl.MappifyUtility.createList(in.reader, writer, "any_value");
+      }
+      nonNullCount.value = 1;
+    }
+
+    @Override
+    public void output() {
+    }
+
+    @Override
+    public void reset() {
+      nonNullCount.value = 0;
+    }
+  }
+  <#elseif aggrtype.funcName.contains("any_value")>
+  @FunctionTemplate(name = "${aggrtype.funcName}",
+                    scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE,
+                    returnType = FunctionTemplate.ReturnType.DECIMAL_AGGREGATE)
+  public static class ${type.inputType}${aggrtype.className} implements DrillAggFunc {
+    @Param ${type.inputType}Holder in;
+    @Inject DrillBuf buffer;
+    @Workspace ObjectHolder value;
+    @Workspace IntHolder scale;
+    @Workspace IntHolder precision;
+    @Output ${type.outputType}Holder out;
+    @Workspace BigIntHolder nonNullCount;
+
+    public void setup() {
+      value = new ObjectHolder();
+      value.obj = java.math.BigDecimal.ZERO;
+      nonNullCount = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      <#if type.inputType?starts_with("Nullable")>
+      sout: {
+        if (in.isSet == 0) {
+        // processing nullable input and the value is null, so don't do anything...
+        break sout;
+      }
+      </#if>
+      if (nonNullCount.value == 0) {
+        value.obj=org.apache.drill.exec.util.DecimalUtility
+            .getBigDecimalFromDrillBuf(in.buffer,in.start,in.end-in.start,in.scale);
+        scale.value = in.scale;
+        precision.value = in.precision;
+      }
+      nonNullCount.value = 1;
+      <#if type.inputType?starts_with("Nullable")>
+      } // end of sout block
+      </#if>
+    }
+
+    @Override
+    public void output() {
+      if (nonNullCount.value > 0) {
+        out.isSet = 1;
+        byte[] bytes = ((java.math.BigDecimal)value.obj).unscaledValue().toByteArray();
+        int len = bytes.length;
+        out.start  = 0;
+        out.buffer = buffer.reallocIfNeeded(len);
+        out.buffer.setBytes(0, bytes);
+        out.end = len;
+        out.scale = scale.value;
+        out.precision = precision.value;
+    } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      scale.value = 0;
+      precision.value = 0;
+      value.obj = null;
+      nonNullCount.value = 0;
+    }
+  }
   <#elseif aggrtype.funcName == "max" || aggrtype.funcName == "min">
 
   @FunctionTemplate(name = "${aggrtype.funcName}",
diff --git a/exec/java-exec/src/main/codegen/templates/SingleValueAgg.java b/exec/java-exec/src/main/codegen/templates/SingleValueAgg.java
new file mode 100644
index 0000000000..c0ff6cf0bd
--- /dev/null
+++ b/exec/java-exec/src/main/codegen/templates/SingleValueAgg.java
@@ -0,0 +1,144 @@
+/*
+ * 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.
+ */
+<@pp.dropOutputFile />
+
+<@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/gaggr/SingleValueFunctions.java" />
+
+<#include "/@includes/license.ftl" />
+
+package org.apache.drill.exec.expr.fn.impl.gaggr;
+
+import org.apache.drill.exec.expr.DrillAggFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.expr.holders.*;
+
+import javax.inject.Inject;
+import io.netty.buffer.DrillBuf;
+
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
+@SuppressWarnings("unused")
+public class SingleValueFunctions {
+<#list singleValue.types as type>
+
+  @FunctionTemplate(name = "single_value",
+                  <#if type.major == "VarDecimal">
+                    returnType = FunctionTemplate.ReturnType.DECIMAL_AVG_AGGREGATE,
+                  </#if>
+                    scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class ${type.inputType}SingleValue implements DrillAggFunc {
+    @Param ${type.inputType}Holder in;
+    @Workspace ${type.runningType}Holder value;
+    @Output ${type.outputType}Holder out;
+    @Workspace BigIntHolder nonNullCount;
+    <#if type.major == "VarDecimal" || type.major == "bytes">
+    @Inject DrillBuf buffer;
+    </#if>
+
+    public void setup() {
+      nonNullCount = new BigIntHolder();
+      nonNullCount.value = 0;
+      value = new ${type.runningType}Holder();
+    }
+
+    @Override
+    public void add() {
+    <#if type.inputType?starts_with("Nullable")>
+      sout: {
+        if (in.isSet == 0) {
+          // processing nullable input and the value is null, so don't do anything...
+          break sout;
+        }
+	  </#if>
+      if (nonNullCount.value == 0) {
+        nonNullCount.value = 1;
+      } else {
+        throw org.apache.drill.common.exceptions.UserException.functionError()
+            .message("Input for single_value function has more than one row")
+            .build();
+      }
+    <#if type.major == "primitive">
+      value.value = in.value;
+    <#elseif type.major == "IntervalDay">
+      value.days = in.days;
+      value.milliseconds = in.milliseconds;
+    <#elseif type.major == "Interval">
+      value.days = in.days;
+      value.milliseconds = in.milliseconds;
+      value.months = in.months;
+    <#elseif type.major == "VarDecimal">
+      value.start = in.start;
+      value.end = in.end;
+      value.buffer = in.buffer;
+      value.scale = in.scale;
+      value.precision = in.precision;
+    <#elseif type.major == "bytes">
+      value.start = in.start;
+      value.end = in.end;
+      value.buffer = in.buffer;
+    </#if>
+    <#if type.inputType?starts_with("Nullable")>
+      } // end of sout block
+	  </#if>
+    }
+
+    @Override
+    public void output() {
+      if (nonNullCount.value > 0) {
+        out.isSet = 1;
+      <#if type.major == "primitive">
+        out.value = value.value;
+      <#elseif type.major == "IntervalDay">
+        out.days = value.days;
+        out.milliseconds = value.milliseconds;
+      <#elseif type.major == "Interval">
+        out.days = value.days;
+        out.milliseconds = value.milliseconds;
+        out.months = value.months;
+      <#elseif type.major == "VarDecimal">
+        out.start = value.start;
+        out.end = value.end;
+        out.buffer = buffer.reallocIfNeeded(value.end - value.start);
+        out.buffer.writeBytes(value.buffer, value.start, value.end - value.start);
+        out.scale = value.scale;
+        out.precision = value.precision;
+      <#elseif type.major == "bytes">
+        out.start = value.start;
+        out.end = value.end;
+        out.buffer = buffer.reallocIfNeeded(value.end - value.start);
+        out.buffer.writeBytes(value.buffer, value.start, value.end - value.start);
+      </#if>
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      value = new ${type.runningType}Holder();
+      nonNullCount.value = 0;
+    }
+  }
+</#list>
+}
+
diff --git a/exec/java-exec/src/main/codegen/templates/VarCharAggrFunctions1.java b/exec/java-exec/src/main/codegen/templates/VarCharAggrFunctions1.java
index a5afce98cf..de5d705e30 100644
--- a/exec/java-exec/src/main/codegen/templates/VarCharAggrFunctions1.java
+++ b/exec/java-exec/src/main/codegen/templates/VarCharAggrFunctions1.java
@@ -90,6 +90,16 @@ public void add() {
         break sout;
       }
     </#if>
+    <#if aggrtype.className == "AnyValue">
+      if (nonNullCount.value == 0) {
+        nonNullCount.value = 1;
+        int inputLength = in.end - in.start;
+        org.apache.drill.exec.expr.fn.impl.DrillByteArray tmp = (org.apache.drill.exec.expr.fn.impl.DrillByteArray) value.obj;
+        byte[] tempArray = new byte[inputLength];
+        in.buffer.getBytes(in.start, tempArray, 0, inputLength);
+        tmp.setBytes(tempArray);
+      }
+    <#else>
     nonNullCount.value = 1;
     org.apache.drill.exec.expr.fn.impl.DrillByteArray tmp = (org.apache.drill.exec.expr.fn.impl.DrillByteArray) value.obj;
     int cmp = 0;
@@ -121,6 +131,7 @@ public void add() {
         tmp.setBytes(tempArray);
       }
     }
+    </#if>
     <#if type.inputType?starts_with("Nullable")>
     } // end of sout block
 	  </#if>
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 1070d762f9..4c840a4139 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
@@ -17,6 +17,7 @@
  */
 package org.apache.drill.exec;
 
+import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.physical.impl.common.HashTable;
 import org.apache.drill.exec.rpc.user.InboundImpersonationManager;
 import org.apache.drill.exec.server.options.OptionValidator;
@@ -84,6 +85,10 @@ private ExecConstants() {
   // need to produce very large batches that take up lot of memory.
   public static final LongValidator OUTPUT_BATCH_SIZE_VALIDATOR = new RangeLongValidator(OUTPUT_BATCH_SIZE, 128, 512 * 1024 * 1024);
 
+  // Based on available memory, adjust output batch size for buffered operators by this factor.
+  public static final String OUTPUT_BATCH_SIZE_AVAIL_MEM_FACTOR = "drill.exec.memory.operator.output_batch_size_avail_mem_factor";
+  public static final DoubleValidator OUTPUT_BATCH_SIZE_AVAIL_MEM_FACTOR_VALIDATOR = new RangeDoubleValidator(OUTPUT_BATCH_SIZE_AVAIL_MEM_FACTOR, 0.01, 1.0);
+
   // External Sort Boot configuration
 
   public static final String EXTERNAL_SORT_TARGET_SPILL_BATCH_SIZE = "drill.exec.sort.external.spill.batch.size";
@@ -204,7 +209,6 @@ private ExecConstants() {
   public static final String SERVICE_KEYTAB_LOCATION = SERVICE_LOGIN_PREFIX + ".keytab";
   public static final String KERBEROS_NAME_MAPPING = SERVICE_LOGIN_PREFIX + ".auth_to_local";
 
-
   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";
   public static final String BIT_ENCRYPTION_SASL_MAX_WRAPPED_SIZE = "drill.exec.security.bit.encryption.sasl.max_wrapped_size";
@@ -316,6 +320,13 @@ private ExecConstants() {
   public static final String PARQUET_FLAT_READER_BULK = "store.parquet.flat.reader.bulk";
   public static final OptionValidator PARQUET_FLAT_READER_BULK_VALIDATOR = new BooleanValidator(PARQUET_FLAT_READER_BULK);
 
+  // Controls the flat parquet reader batching constraints (number of record and memory limit)
+  public static final String PARQUET_FLAT_BATCH_NUM_RECORDS = "store.parquet.flat.batch.num_records";
+  public static final OptionValidator PARQUET_FLAT_BATCH_NUM_RECORDS_VALIDATOR = new RangeLongValidator(PARQUET_FLAT_BATCH_NUM_RECORDS, 1, Integer.MAX_VALUE);
+  public static final String PARQUET_FLAT_BATCH_MEMORY_SIZE = "store.parquet.flat.batch.memory_size";
+  // This configuration is used to overwrite the common memory batch sizing configuration property
+  public static final OptionValidator PARQUET_FLAT_BATCH_MEMORY_SIZE_VALIDATOR = new RangeLongValidator(PARQUET_FLAT_BATCH_MEMORY_SIZE, 0, Integer.MAX_VALUE);
+
   public static final String JSON_ALL_TEXT_MODE = "store.json.all_text_mode";
   public static final BooleanValidator JSON_READER_ALL_TEXT_MODE_VALIDATOR = new BooleanValidator(JSON_ALL_TEXT_MODE);
   public static final BooleanValidator JSON_EXTENDED_TYPES = new BooleanValidator("store.json.extended_types");
@@ -387,9 +398,20 @@ private ExecConstants() {
 
   // TODO: We need to add a feature that enables storage plugins to add their own options. Currently we have to declare
   // in core which is not right. Move this option and above two mongo plugin related options once we have the feature.
+  @Deprecated // TODO: DRILL-6527. It should be removed starting from next Drill 1.15.0 release
   public static final String HIVE_OPTIMIZE_SCAN_WITH_NATIVE_READERS = "store.hive.optimize_scan_with_native_readers";
+  @Deprecated // TODO: DRILL-6527. It should be removed starting from next Drill 1.15.0 release
   public static final OptionValidator HIVE_OPTIMIZE_SCAN_WITH_NATIVE_READERS_VALIDATOR =
       new BooleanValidator(HIVE_OPTIMIZE_SCAN_WITH_NATIVE_READERS);
+  public static final String HIVE_OPTIMIZE_PARQUET_SCAN_WITH_NATIVE_READER = "store.hive.parquet.optimize_scan_with_native_reader";
+  public static final OptionValidator HIVE_OPTIMIZE_PARQUET_SCAN_WITH_NATIVE_READER_VALIDATOR =
+      new BooleanValidator(HIVE_OPTIMIZE_PARQUET_SCAN_WITH_NATIVE_READER);
+  public static final String HIVE_OPTIMIZE_MAPRDB_JSON_SCAN_WITH_NATIVE_READER = "store.hive.maprdb_json.optimize_scan_with_native_reader";
+  public static final OptionValidator HIVE_OPTIMIZE_MAPRDB_JSON_SCAN_WITH_NATIVE_READER_VALIDATOR =
+      new BooleanValidator(HIVE_OPTIMIZE_MAPRDB_JSON_SCAN_WITH_NATIVE_READER);
+
+  public static final String HIVE_CONF_PROPERTIES = "store.hive.conf.properties";
+  public static final OptionValidator HIVE_CONF_PROPERTIES_VALIDATOR = new StringValidator(HIVE_CONF_PROPERTIES);
 
   public static final String SLICE_TARGET = "planner.slice_target";
   public static final long SLICE_TARGET_DEFAULT = 100000l;
@@ -441,7 +463,7 @@ private ExecConstants() {
    * DEFAULT: 2048 MB
    */
   public static final String MAX_QUERY_MEMORY_PER_NODE_KEY = "planner.memory.max_query_memory_per_node";
-  public static final LongValidator MAX_QUERY_MEMORY_PER_NODE = new RangeLongValidator(MAX_QUERY_MEMORY_PER_NODE_KEY, 1024 * 1024, Long.MAX_VALUE);
+  public static final LongValidator MAX_QUERY_MEMORY_PER_NODE = new RangeLongValidator(MAX_QUERY_MEMORY_PER_NODE_KEY, 1024 * 1024, DrillConfig.getMaxDirectMemory());
 
   /**
    * Alternative way to compute per-query-per-node memory as a percent
@@ -681,4 +703,13 @@ public static String bootDefaultFor(String name) {
 
   public static final String ALLOW_LOOPBACK_ADDRESS_BINDING = "drill.exec.allow_loopback_address_binding";
 
+  /** Enables batch size statistics logging */
+  public static final String STATS_LOGGING_BATCH_SIZE_OPTION = "drill.exec.stats.logging.batch_size";
+  public static final BooleanValidator STATS_LOGGING_BATCH_SIZE_VALIDATOR = new BooleanValidator(STATS_LOGGING_BATCH_SIZE_OPTION);
+
+  /** Enables fine-grained batch size statistics logging */
+  public static final String STATS_LOGGING_FG_BATCH_SIZE_OPTION = "drill.exec.stats.logging.fine_grained.batch_size";
+  public static final BooleanValidator STATS_LOGGING_BATCH_FG_SIZE_VALIDATOR = new BooleanValidator(STATS_LOGGING_FG_BATCH_SIZE_OPTION);
+
+
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java b/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
index f880b936c3..ccd7e36361 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
@@ -520,7 +520,7 @@ public RpcEndpointInfos getServerInfos() {
    *
    * @return the server name, or null if not connected or if the server
    *         doesn't provide the name
-   * @return
+   * @return The server name.
    */
   public String getServerName() {
     return (client != null && client.getServerInfos() != null) ? client.getServerInfos().getName() : null;
@@ -533,7 +533,7 @@ public String getServerName() {
    *
    * @return the server version, or null if not connected or if the server
    *         doesn't provide the version
-   * @return
+   * @return The server version.
    */
   public Version getServerVersion() {
     return (client != null && client.getServerInfos() != null) ? UserRpcUtils.getVersion(client.getServerInfos()) : null;
@@ -588,7 +588,7 @@ public Version getServerVersion() {
    * @param type
    * @param query
    * @param isSplitPlan - option to tell whether to return single or split plans for a query
-   * @return list of PlanFragments that can be used later on in {@link #runQuery(QueryType, List, UserResultsListener)}
+   * @return list of PlanFragments that can be used later on in {@link #runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType, java.util.List, org.apache.drill.exec.rpc.user.UserResultsListener)}
    * to run a query without additional planning
    */
   public DrillRpcFuture<QueryPlanFragments> planQuery(QueryType type, String query, boolean isSplitPlan) {
@@ -663,7 +663,7 @@ public void runQuery(QueryType type, List<PlanFragment> planFragments, UserResul
    * Get the list of catalogs in <code>INFORMATION_SCHEMA.CATALOGS</code> table satisfying the given filters.
    *
    * @param catalogNameFilter Filter on <code>catalog name</code>. Pass null to apply no filter.
-   * @return
+   * @return The list of catalogs in <code>INFORMATION_SCHEMA.CATALOGS</code> table satisfying the given filters.
    */
   public DrillRpcFuture<GetCatalogsResp> getCatalogs(LikeFilter catalogNameFilter) {
     final GetCatalogsReq.Builder reqBuilder = GetCatalogsReq.newBuilder();
@@ -679,7 +679,7 @@ public void runQuery(QueryType type, List<PlanFragment> planFragments, UserResul
    *
    * @param catalogNameFilter Filter on <code>catalog name</code>. Pass null to apply no filter.
    * @param schemaNameFilter Filter on <code>schema name</code>. Pass null to apply no filter.
-   * @return
+   * @return The list of schemas in <code>INFORMATION_SCHEMA.SCHEMATA</code> table satisfying the given filters.
    */
   public DrillRpcFuture<GetSchemasResp> getSchemas(LikeFilter catalogNameFilter, LikeFilter schemaNameFilter) {
     final GetSchemasReq.Builder reqBuilder = GetSchemasReq.newBuilder();
@@ -701,7 +701,7 @@ public void runQuery(QueryType type, List<PlanFragment> planFragments, UserResul
    * @param schemaNameFilter Filter on <code>schema name</code>. Pass null to apply no filter.
    * @param tableNameFilter Filter in <code>table name</code>. Pass null to apply no filter.
    * @param tableTypeFilter Filter in <code>table type</code>. Pass null to apply no filter
-   * @return
+   * @return The list of tables in <code>INFORMATION_SCHEMA.TABLES</code> table satisfying the given filters.
    */
   public DrillRpcFuture<GetTablesResp> getTables(LikeFilter catalogNameFilter, LikeFilter schemaNameFilter,
       LikeFilter tableNameFilter, List<String> tableTypeFilter) {
@@ -732,7 +732,7 @@ public void runQuery(QueryType type, List<PlanFragment> planFragments, UserResul
    * @param schemaNameFilter Filter on <code>schema name</code>. Pass null to apply no filter.
    * @param tableNameFilter Filter in <code>table name</code>. Pass null to apply no filter.
    * @param columnNameFilter Filter in <code>column name</code>. Pass null to apply no filter.
-   * @return
+   * @return The list of columns in <code>INFORMATION_SCHEMA.COLUMNS</code> table satisfying the given filters.
    */
   public DrillRpcFuture<GetColumnsResp> getColumns(LikeFilter catalogNameFilter, LikeFilter schemaNameFilter,
       LikeFilter tableNameFilter, LikeFilter columnNameFilter) {
@@ -757,10 +757,10 @@ public void runQuery(QueryType type, List<PlanFragment> planFragments, UserResul
   }
 
   /**
-   * Create a prepared statement for given <code>query</code>.
+   * Create a prepared statement for given the <code>query</code>.
    *
    * @param query
-   * @return
+   * @return The prepared statement for given the <code>query</code>.
    */
   public DrillRpcFuture<CreatePreparedStatementResp> createPreparedStatement(final String query) {
     final CreatePreparedStatementReq req =
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java
index d7646633d6..0175d51a9c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java
@@ -22,6 +22,7 @@
 import java.util.List;
 import java.util.Set;
 
+import org.apache.drill.common.expression.AnyValueExpression;
 import org.apache.drill.common.expression.BooleanOperator;
 import org.apache.drill.common.expression.CastExpression;
 import org.apache.drill.common.expression.ConvertExpression;
@@ -234,6 +235,12 @@ public Boolean visitConvertExpression(ConvertExpression e,
     return e.getInput().accept(this, value);
   }
 
+  @Override
+  public Boolean visitAnyValueExpression(AnyValueExpression e,
+                                        IdentityHashMap<LogicalExpression, Object> value) throws RuntimeException {
+    return e.getInput().accept(this, value);
+  }
+
   @Override
   public Boolean visitParameter(ValueExpressions.ParameterExpression e, IdentityHashMap<LogicalExpression, Object> value) throws RuntimeException {
     return false;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/AbstractExecExprVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/AbstractExecExprVisitor.java
new file mode 100644
index 0000000000..36e2bbadcd
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/AbstractExecExprVisitor.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.expr;
+
+import org.apache.drill.common.expression.visitors.AbstractExprVisitor;
+import org.apache.drill.exec.physical.impl.project.OutputWidthExpression;
+import org.apache.drill.exec.physical.impl.project.OutputWidthExpression.FunctionCallExpr;
+import org.apache.drill.exec.physical.impl.project.OutputWidthExpression.FixedLenExpr;
+import org.apache.drill.exec.physical.impl.project.OutputWidthExpression.VarLenReadExpr;
+import org.apache.drill.exec.physical.impl.project.OutputWidthExpression.IfElseWidthExpr;
+
+
+public abstract class AbstractExecExprVisitor<T, VAL, EXCEP extends Exception> extends AbstractExprVisitor<T, VAL, EXCEP> {
+
+    public T visitValueVectorWriteExpression(ValueVectorWriteExpression writeExpr, VAL value) throws EXCEP {
+        return visitUnknown(writeExpr, value);
+    }
+
+    public T visitValueVectorReadExpression(ValueVectorReadExpression readExpr, VAL value) throws EXCEP {
+        return visitUnknown(readExpr, value);
+    }
+
+    public T visitFunctionCallExpr(FunctionCallExpr functionCallExpr, VAL value) throws EXCEP {
+        return visitUnknown(functionCallExpr, value);
+    }
+
+    public T visitFixedLenExpr(FixedLenExpr fixedLenExpr, VAL value) throws EXCEP {
+        return visitUnknown(fixedLenExpr, value);
+    }
+
+    public T visitVarLenReadExpr(VarLenReadExpr varLenReadExpr, VAL value) throws EXCEP {
+        return visitUnknown(varLenReadExpr, value);
+    }
+
+    public T visitIfElseWidthExpr(IfElseWidthExpr ifElseWidthExpr, VAL value) throws EXCEP {
+        return visitUnknown(ifElseWidthExpr, value);
+    }
+
+    public T visitUnknown(OutputWidthExpression e, VAL value) throws EXCEP {
+        throw new UnsupportedOperationException(String.format("Expression of type %s not handled by visitor type %s.",
+                e.getClass().getCanonicalName(), this.getClass().getCanonicalName()));
+    }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/CodeGenerator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/CodeGenerator.java
index 480b2c2ea3..76d68e22df 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/CodeGenerator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/CodeGenerator.java
@@ -46,7 +46,7 @@
  * class without the normal byte-code manipulations. Plain Java allows
  * the option to persist, and debug, the generated code when building new
  * generated classes or otherwise working with generated code. To turn
- * on debugging, see the explanation in {@link ClassBuilder}.
+ * on debugging, see the explanation in {@link org.apache.drill.exec.compile.ClassBuilder}.
  *
  * @param <T>
  *          The interface that results from compiling and merging the runtime
@@ -149,7 +149,7 @@ public boolean isPlainJava() {
   /**
    * Debug-time option to persist the code for the generated class to permit debugging.
    * Has effect only when code is generated using the plain Java option. Code
-   * is written to the code directory specified in {@link ClassBuilder}.
+   * is written to the code directory specified in {@link org.apache.drill.exec.compile.ClassBuilder}.
    * To debug code, set this option, then point your IDE to the code directory
    * when the IDE prompts you for the source code location.
    *
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
index 64cfe66953..4486972ee1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
@@ -24,6 +24,7 @@
 import java.util.Set;
 import java.util.Stack;
 
+import org.apache.drill.common.expression.AnyValueExpression;
 import org.apache.drill.common.expression.BooleanOperator;
 import org.apache.drill.common.expression.CastExpression;
 import org.apache.drill.common.expression.ConvertExpression;
@@ -181,7 +182,7 @@ public HoldingContainer visitFunctionHolderExpression(FunctionHolderExpression h
 
       AbstractFuncHolder holder = (AbstractFuncHolder) holderExpr.getHolder();
 
-      JVar[] workspaceVars = holder.renderStart(generator, null);
+      JVar[] workspaceVars = holder.renderStart(generator, null, holderExpr.getFieldReference());
 
       if (holder.isNested()) {
         generator.getMappingSet().enterChild();
@@ -456,8 +457,7 @@ private HoldingContainer visitValueVectorReadExpression(ValueVectorReadExpressio
         generator.getEvalBlock().add(eval);
 
       } else {
-        JExpression vector = e.isSuperReader() ? vv1.component(componentVariable) : vv1;
-        JExpression expr = vector.invoke("getReader");
+        JExpression expr = vv1.invoke("getReader");
         PathSegment seg = e.getReadPath();
 
         JVar isNull = null;
@@ -713,6 +713,17 @@ public HoldingContainer visitConvertExpression(ConvertExpression e, ClassGenerat
       return fc.accept(this, value);
     }
 
+    @Override
+    public HoldingContainer visitAnyValueExpression(AnyValueExpression e, ClassGenerator<?> value)
+        throws RuntimeException {
+
+      List<LogicalExpression> newArgs = Lists.newArrayList();
+      newArgs.add(e.getInput()); // input_expr
+
+      FunctionCall fc = new FunctionCall(AnyValueExpression.ANY_VALUE, newArgs, e.getPosition());
+      return fc.accept(this, value);
+    }
+
     private HoldingContainer visitBooleanAnd(BooleanOperator op,
         ClassGenerator<?> generator) {
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorReadExpression.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorReadExpression.java
index 410c48aefa..370960be27 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorReadExpression.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorReadExpression.java
@@ -17,8 +17,6 @@
  */
 package org.apache.drill.exec.expr;
 
-import java.util.Iterator;
-
 import com.google.common.collect.ImmutableSet;
 import org.apache.drill.common.expression.ExpressionPosition;
 import org.apache.drill.common.expression.LogicalExpression;
@@ -27,6 +25,8 @@
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.exec.record.TypedFieldId;
 
+import java.util.Iterator;
+
 /**
  * Wraps a value vector field to be read, providing metadata about the field.
  * Also may contain batch naming information to which this field belongs.
@@ -73,6 +73,10 @@ public MajorType getMajorType() {
 
   @Override
   public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E> visitor, V value) throws E {
+    if (visitor instanceof AbstractExecExprVisitor) {
+      AbstractExecExprVisitor<T, V, E> abstractExecExprVisitor = (AbstractExecExprVisitor<T, V, E>) visitor;
+      return abstractExecExprVisitor.visitValueVectorReadExpression(this, value);
+    }
     return visitor.visitUnknown(this, value);
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorWriteExpression.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorWriteExpression.java
index e2a947bda5..f419c88910 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorWriteExpression.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorWriteExpression.java
@@ -17,8 +17,7 @@
  */
 package org.apache.drill.exec.expr;
 
-import java.util.Iterator;
-
+import com.google.common.collect.Iterators;
 import org.apache.drill.common.expression.ExpressionPosition;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.expression.visitors.ExprVisitor;
@@ -26,7 +25,7 @@
 import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.record.TypedFieldId;
 
-import com.google.common.collect.Iterators;
+import java.util.Iterator;
 
 public class ValueVectorWriteExpression implements LogicalExpression {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ValueVectorWriteExpression.class);
@@ -61,6 +60,10 @@ public boolean isSafe() {
 
   @Override
   public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E> visitor, V value) throws E {
+    if (visitor instanceof AbstractExecExprVisitor) {
+      AbstractExecExprVisitor<T, V, E> abstractExecExprVisitor = (AbstractExecExprVisitor<T, V, E>) visitor;
+      return abstractExecExprVisitor.visitValueVectorWriteExpression(this, value);
+    }
     return visitor.visitUnknown(this, value);
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/FunctionTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/FunctionTemplate.java
index 1a5470ba20..11914ea9fd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/FunctionTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/FunctionTemplate.java
@@ -23,6 +23,8 @@
 import org.apache.drill.exec.expr.fn.output.ConcatReturnTypeInference;
 import org.apache.drill.exec.expr.fn.output.DecimalReturnTypeInference;
 import org.apache.drill.exec.expr.fn.output.DefaultReturnTypeInference;
+import org.apache.drill.exec.expr.fn.output.OutputWidthCalculator;
+import org.apache.drill.exec.expr.fn.output.OutputWidthCalculators;
 import org.apache.drill.exec.expr.fn.output.PadReturnTypeInference;
 import org.apache.drill.exec.expr.fn.output.ReturnTypeInference;
 import org.apache.drill.exec.expr.fn.output.SameInOutLengthReturnTypeInference;
@@ -88,6 +90,38 @@
   boolean isNiladic() default false;
   boolean checkPrecisionRange() default false;
 
+  /**
+   * This enum will be used to estimate the average size of the output
+   * produced by a function that produces variable length output
+   */
+  enum OutputWidthCalculatorType {
+    DEFAULT(OutputWidthCalculators.DefaultOutputWidthCalculator.INSTANCE),
+    CLONE(OutputWidthCalculators.CloneOutputWidthCalculator.INSTANCE),
+    CONCAT(OutputWidthCalculators.ConcatOutputWidthCalculator.INSTANCE),
+    // Custom calculator are required for functions that don't fall in to any pre-defined
+    // calculator categories - like replace and lpad
+    // place holder markers on functions until support
+    // for CUSTOM calculators is implemented
+    // CUSTOM_FIXED_WIDTH_DEFUALT will default to a fixed size - for functions like
+    // lpad() where the ouput size does not easily map to the input size
+    CUSTOM_FIXED_WIDTH_DEFUALT(OutputWidthCalculators.DefaultOutputWidthCalculator.INSTANCE),
+    // CUSTOM CLONE will default to CLONE - for functions like replace() where the output
+    // size  does not easily map to the input size but is likely to be at most the size of the input.
+    CUSTOM_CLONE_DEFAULT(OutputWidthCalculators.CloneOutputWidthCalculator.INSTANCE);
+    OutputWidthCalculator outputWidthCalculator;
+
+    OutputWidthCalculatorType(OutputWidthCalculator outputWidthCalculator) {
+      this.outputWidthCalculator = outputWidthCalculator;
+    }
+
+    public OutputWidthCalculator getOutputWidthCalculator() { return outputWidthCalculator; }
+  }
+
+  OutputWidthCalculatorType outputWidthCalculatorType() default OutputWidthCalculatorType.DEFAULT;
+
+  int OUTPUT_SIZE_ESTIMATE_DEFAULT = -1;
+  int outputSizeEstimate() default OUTPUT_SIZE_ESTIMATE_DEFAULT;
+
   enum NullHandling {
     /**
      * Method handles nulls.
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/AbstractFuncHolder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/AbstractFuncHolder.java
index 4902260f2c..7dd58ace24 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/AbstractFuncHolder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/AbstractFuncHolder.java
@@ -32,7 +32,7 @@
 
 public abstract class AbstractFuncHolder implements FuncHolder {
 
-  public abstract JVar[] renderStart(ClassGenerator<?> g, HoldingContainer[] inputVariables);
+  public abstract JVar[] renderStart(ClassGenerator<?> g, HoldingContainer[] inputVariables, FieldReference fieldReference);
 
   public void renderMiddle(ClassGenerator<?> g, HoldingContainer[] inputVariables, JVar[] workspaceJVars) {
     // default implementation is add no code
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillAggFuncHolder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillAggFuncHolder.java
index e1cd96fefb..1a5df670f6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillAggFuncHolder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillAggFuncHolder.java
@@ -21,6 +21,7 @@
 
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.expression.FieldReference;
+import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.Types;
@@ -44,19 +45,19 @@
 class DrillAggFuncHolder extends DrillFuncHolder {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillAggFuncHolder.class);
 
-  private String setup() {
+  protected String setup() {
     return meth("setup");
   }
-  private String reset() {
+  protected String reset() {
     return meth("reset", false);
   }
-  private String add() {
+  protected String add() {
     return meth("add");
   }
-  private String output() {
+  protected String output() {
     return meth("output");
   }
-  private String cleanup() {
+  protected String cleanup() {
     return meth("cleanup", false);
   }
 
@@ -78,7 +79,7 @@ public boolean isAggregating() {
   }
 
   @Override
-  public JVar[] renderStart(ClassGenerator<?> g, HoldingContainer[] inputVariables) {
+  public JVar[] renderStart(ClassGenerator<?> g, HoldingContainer[] inputVariables, FieldReference fieldReference) {
     if (!g.getMappingSet().isHashAggMapping()) {  //Declare workspace vars for non-hash-aggregation.
         JVar[] workspaceJVars = declareWorkspaceVariables(g);
         generateBody(g, BlockType.SETUP, setup(), null, workspaceJVars, true);
@@ -128,12 +129,20 @@ public void renderMiddle(ClassGenerator<?> g, HoldingContainer[] inputVariables,
   @Override
   public HoldingContainer renderEnd(ClassGenerator<?> classGenerator, HoldingContainer[] inputVariables,
                                     JVar[] workspaceJVars, FieldReference fieldReference) {
-    HoldingContainer out = classGenerator.declare(getReturnType(), false);
+    HoldingContainer out = null;
+    JVar internalOutput = null;
+    if (getReturnType().getMinorType() != TypeProtos.MinorType.LATE) {
+      out = classGenerator.declare(getReturnType(), false);
+    }
     JBlock sub = new JBlock();
+    if (getReturnType().getMinorType() != TypeProtos.MinorType.LATE) {
+      internalOutput = sub.decl(JMod.FINAL, classGenerator.getHolderType(getReturnType()), getReturnValue().getName(), JExpr._new(classGenerator.getHolderType(getReturnType())));
+    }
     classGenerator.getEvalBlock().add(sub);
-    JVar internalOutput = sub.decl(JMod.FINAL, classGenerator.getHolderType(getReturnType()), getReturnValue().getName(), JExpr._new(classGenerator.getHolderType(getReturnType())));
     addProtectedBlock(classGenerator, sub, output(), null, workspaceJVars, false);
-    sub.assign(out.getHolder(), internalOutput);
+    if (getReturnType().getMinorType() != TypeProtos.MinorType.LATE) {
+      sub.assign(out.getHolder(), internalOutput);
+    }
     //hash aggregate uses workspace vectors. Initialization is done in "setup" and does not require "reset" block.
     if (!classGenerator.getMappingSet().isHashAggMapping()) {
       generateBody(classGenerator, BlockType.RESET, reset(), null, workspaceJVars, false);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillComplexWriterAggFuncHolder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillComplexWriterAggFuncHolder.java
new file mode 100644
index 0000000000..44766bdbb3
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillComplexWriterAggFuncHolder.java
@@ -0,0 +1,142 @@
+/*
+ * 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.fn;
+
+import org.apache.drill.common.expression.FieldReference;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.expr.ClassGenerator;
+import org.apache.drill.exec.expr.ClassGenerator.HoldingContainer;
+import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
+import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
+import org.apache.drill.exec.record.VectorAccessibleComplexWriter;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter.ComplexWriter;
+
+import com.sun.codemodel.JBlock;
+import com.sun.codemodel.JClass;
+import com.sun.codemodel.JExpr;
+import com.sun.codemodel.JExpression;
+import com.sun.codemodel.JInvocation;
+import com.sun.codemodel.JVar;
+import com.sun.codemodel.JMod;
+
+public class DrillComplexWriterAggFuncHolder extends DrillAggFuncHolder {
+
+  // Complex writer to write out complex data-types e.g. repeated maps/lists
+  private JVar complexWriter;
+  // The index at which to write - important when group-by is present. Implicit assumption that the output indexes
+  // will be sequential starting from 0. i.e. the first group would be written at index 0, second group at index 1
+  // and so on.
+  private JVar writerIdx;
+  private JVar lastWriterIdx;
+  public DrillComplexWriterAggFuncHolder(FunctionAttributes functionAttributes, FunctionInitializer initializer) {
+    super(functionAttributes, initializer);
+  }
+
+  @Override
+  public boolean isComplexWriterFuncHolder() {
+    return true;
+  }
+
+  @Override
+  public JVar[] renderStart(ClassGenerator<?> classGenerator, HoldingContainer[] inputVariables, FieldReference fieldReference) {
+    if (!classGenerator.getMappingSet().isHashAggMapping()) {  //Declare workspace vars for non-hash-aggregation.
+      JInvocation container = classGenerator.getMappingSet().getOutgoing().invoke("getOutgoingContainer");
+
+      complexWriter = classGenerator.declareClassField("complexWriter", classGenerator.getModel()._ref(ComplexWriter.class));
+      writerIdx = classGenerator.declareClassField("writerIdx", classGenerator.getModel()._ref(int.class));
+      lastWriterIdx = classGenerator.declareClassField("lastWriterIdx", classGenerator.getModel()._ref(int.class));
+      //Default name is "col", if not passed in a reference name for the output vector.
+      String refName = fieldReference == null ? "col" : fieldReference.getRootSegment().getPath();
+      JClass cwClass = classGenerator.getModel().ref(VectorAccessibleComplexWriter.class);
+      classGenerator.getSetupBlock().assign(complexWriter, cwClass.staticInvoke("getWriter").arg(refName).arg(container));
+      classGenerator.getSetupBlock().assign(writerIdx, JExpr.lit(0));
+      classGenerator.getSetupBlock().assign(lastWriterIdx, JExpr.lit(-1));
+
+      JVar[] workspaceJVars = declareWorkspaceVariables(classGenerator);
+      generateBody(classGenerator, ClassGenerator.BlockType.SETUP, setup(), null, workspaceJVars, true);
+      return workspaceJVars;
+    } else {
+      return super.renderStart(classGenerator, inputVariables, fieldReference);
+    }
+  }
+
+  @Override
+  public void renderMiddle(ClassGenerator<?> classGenerator, HoldingContainer[] inputVariables, JVar[] workspaceJVars) {
+
+    classGenerator.getEvalBlock().directStatement(String.format("//---- start of eval portion of %s function. ----//",
+        getRegisteredNames()[0]));
+
+    JBlock sub = new JBlock(true, true);
+    JBlock topSub = sub;
+    JClass aggBatchClass = null;
+
+    if (classGenerator.getCodeGenerator().getDefinition() == StreamingAggTemplate.TEMPLATE_DEFINITION) {
+      aggBatchClass = classGenerator.getModel().ref(StreamingAggBatch.class);
+    }
+    assert aggBatchClass != null : "ComplexWriterAggFuncHolder should only be used with an Aggregate Operator";
+
+    JExpression aggBatch = JExpr.cast(aggBatchClass, classGenerator.getMappingSet().getOutgoing());
+
+    classGenerator.getSetupBlock().add(aggBatch.invoke("addComplexWriter").arg(complexWriter));
+    // Only set the writer if there is a position change. Calling setPosition may cause underlying writers to allocate
+    // new vectors, thereby, losing the previously stored values
+    JBlock condAssignCW = classGenerator.getEvalBlock()._if(lastWriterIdx.ne(writerIdx))._then();
+    condAssignCW.add(complexWriter.invoke("setPosition").arg(writerIdx));
+    condAssignCW.assign(lastWriterIdx, writerIdx);
+    sub.decl(classGenerator.getModel()._ref(ComplexWriter.class), getReturnValue().getName(), complexWriter);
+
+    // add the subblock after the out declaration.
+    classGenerator.getEvalBlock().add(topSub);
+
+    addProtectedBlock(classGenerator, sub, add(), inputVariables, workspaceJVars, false);
+    classGenerator.getEvalBlock().directStatement(String.format("//---- end of eval portion of %s function. ----//",
+        getRegisteredNames()[0]));
+  }
+
+  @Override
+  public HoldingContainer renderEnd(ClassGenerator<?> classGenerator, HoldingContainer[] inputVariables,
+                                    JVar[] workspaceJVars, FieldReference fieldReference) {
+    HoldingContainer out = null;
+    JVar internalOutput = null;
+    if (getReturnType().getMinorType() != TypeProtos.MinorType.LATE) {
+      out = classGenerator.declare(getReturnType(), false);
+    }
+    JBlock sub = new JBlock();
+    if (getReturnType().getMinorType() != TypeProtos.MinorType.LATE) {
+      internalOutput = sub.decl(JMod.FINAL, classGenerator.getHolderType(getReturnType()), getReturnValue().getName(),
+          JExpr._new(classGenerator.getHolderType(getReturnType())));
+    }
+    classGenerator.getEvalBlock().add(sub);
+    if (getReturnType().getMinorType() == TypeProtos.MinorType.LATE) {
+      sub.assignPlus(writerIdx, JExpr.lit(1));
+    }
+    addProtectedBlock(classGenerator, sub, output(), null, workspaceJVars, false);
+    if (getReturnType().getMinorType() != TypeProtos.MinorType.LATE) {
+      sub.assign(out.getHolder(), internalOutput);
+    }
+    //hash aggregate uses workspace vectors. Initialization is done in "setup" and does not require "reset" block.
+    if (!classGenerator.getMappingSet().isHashAggMapping()) {
+      generateBody(classGenerator, ClassGenerator.BlockType.RESET, reset(), null, workspaceJVars, false);
+    }
+    generateBody(classGenerator, ClassGenerator.BlockType.CLEANUP, cleanup(), null, workspaceJVars, false);
+
+    return out;
+  }
+}
+
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFuncHolder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFuncHolder.java
index 9df5305125..b169fb1b73 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFuncHolder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFuncHolder.java
@@ -17,14 +17,19 @@
  */
 package org.apache.drill.exec.expr.fn;
 
-import java.util.Arrays;
-import java.util.List;
-
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import com.sun.codemodel.JBlock;
+import com.sun.codemodel.JExpr;
+import com.sun.codemodel.JType;
+import com.sun.codemodel.JVar;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.expression.ExpressionPosition;
+import org.apache.drill.common.expression.FieldReference;
 import org.apache.drill.common.expression.FunctionHolderExpression;
 import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.common.types.Types;
@@ -36,15 +41,15 @@
 import org.apache.drill.exec.expr.DrillFuncHolderExpr;
 import org.apache.drill.exec.expr.TypeHelper;
 import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
+import org.apache.drill.exec.expr.fn.output.OutputWidthCalculator;
+import org.apache.drill.exec.expr.holders.ListHolder;
+import org.apache.drill.exec.expr.holders.MapHolder;
+import org.apache.drill.exec.expr.holders.RepeatedMapHolder;
 import org.apache.drill.exec.ops.UdfUtilities;
 import org.apache.drill.exec.vector.complex.reader.FieldReader;
 
-import com.google.common.base.Preconditions;
-import com.google.common.base.Strings;
-import com.sun.codemodel.JBlock;
-import com.sun.codemodel.JExpr;
-import com.sun.codemodel.JType;
-import com.sun.codemodel.JVar;
+import java.util.Arrays;
+import java.util.List;
 
 public abstract class DrillFuncHolder extends AbstractFuncHolder {
 
@@ -80,7 +85,7 @@ protected String meth(String methodName, boolean required) {
   }
 
   @Override
-  public JVar[] renderStart(ClassGenerator<?> g, HoldingContainer[] inputVariables) {
+  public JVar[] renderStart(ClassGenerator<?> g, HoldingContainer[] inputVariables, FieldReference fieldReference) {
     return declareWorkspaceVariables(g);
   }
 
@@ -101,6 +106,7 @@ public boolean isNiladic() {
     return attributes.isNiladic();
   }
 
+
   /**
    * Generates string representation of function input parameters:
    * PARAMETER_TYPE_1-PARAMETER_MODE_1,PARAMETER_TYPE_2-PARAMETER_MODE_2
@@ -186,12 +192,35 @@ protected void addProtectedBlock(ClassGenerator<?> g, JBlock sub, String body, H
 
         ValueReference parameter = attributes.getParameters()[i];
         HoldingContainer inputVariable = inputVariables[i];
-        if (parameter.isFieldReader() && ! inputVariable.isReader() && ! Types.isComplex(inputVariable.getMajorType()) && inputVariable.getMinorType() != MinorType.UNION) {
+        if (parameter.isFieldReader() && ! inputVariable.isReader()
+            && ! Types.isComplex(inputVariable.getMajorType()) && inputVariable.getMinorType() != MinorType.UNION) {
           JType singularReaderClass = g.getModel()._ref(TypeHelper.getHolderReaderImpl(inputVariable.getMajorType().getMinorType(),
               inputVariable.getMajorType().getMode()));
           JType fieldReadClass = g.getModel()._ref(FieldReader.class);
           sub.decl(fieldReadClass, parameter.getName(), JExpr._new(singularReaderClass).arg(inputVariable.getHolder()));
-        } else {
+        } else if (!parameter.isFieldReader() && inputVariable.isReader() && Types.isComplex(parameter.getType())) {
+          // For complex data-types (repeated maps/lists) the input to the aggregate will be a FieldReader. However, aggregate
+          // functions like ANY_VALUE, will assume the input to be a RepeatedMapHolder etc. Generate boilerplate code, to map
+          // from FieldReader to respective Holder.
+          if (parameter.getType().getMinorType() == MinorType.MAP) {
+            JType holderClass;
+            if (parameter.getType().getMode() == TypeProtos.DataMode.REPEATED) {
+              holderClass = g.getModel()._ref(RepeatedMapHolder.class);
+              JVar holderVar = sub.decl(holderClass, parameter.getName(), JExpr._new(holderClass));
+              sub.assign(holderVar.ref("reader"), inputVariable.getHolder());
+            } else {
+              holderClass = g.getModel()._ref(MapHolder.class);
+              JVar holderVar = sub.decl(holderClass, parameter.getName(), JExpr._new(holderClass));
+              sub.assign(holderVar.ref("reader"), inputVariable.getHolder());
+            }
+          } else if (parameter.getType().getMinorType() == MinorType.LIST) {
+            //TODO: Add support for REPEATED LISTs
+            JType holderClass = g.getModel()._ref(ListHolder.class);
+            JVar holderVar = sub.decl(holderClass, parameter.getName(), JExpr._new(holderClass));
+            sub.assign(holderVar.ref("reader"), inputVariable.getHolder());
+          }
+        }
+        else {
           sub.decl(inputVariable.getHolder().type(), parameter.getName(), inputVariable.getHolder());
         }
       }
@@ -263,6 +292,14 @@ public MajorType getReturnType(final List<LogicalExpression> logicalExpressions)
     return attributes.getReturnType().getType(logicalExpressions, attributes);
   }
 
+  public OutputWidthCalculator getOutputWidthCalculator() {
+    return attributes.getOutputWidthCalculatorType().getOutputWidthCalculator();
+  }
+
+  public int variableOutputSizeEstimate(){
+    return attributes.variableOutputSizeEstimate();
+  }
+
   public NullHandling getNullHandling() {
     return attributes.getNullHandling();
   }
@@ -309,6 +346,4 @@ public String toString() {
         + ", parameters=" + (attributes.getParameters() != null ?
         Arrays.asList(attributes.getParameters()).subList(0, Math.min(attributes.getParameters().length, maxLen)) : null) + "]";
   }
-
-
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionAttributes.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionAttributes.java
index 4fd5be77b7..6d1b767828 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionAttributes.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionAttributes.java
@@ -54,6 +54,14 @@ public FunctionScope getScope() {
     return template.returnType();
   }
 
+  public FunctionTemplate.OutputWidthCalculatorType getOutputWidthCalculatorType() {
+    return template.outputWidthCalculatorType();
+  }
+
+  public int variableOutputSizeEstimate() {
+    return  template.outputSizeEstimate();
+  }
+
   public NullHandling getNullHandling() {
     return template.nulls();
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionConverter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionConverter.java
index ca5605a582..b5a2f0700d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionConverter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionConverter.java
@@ -181,7 +181,9 @@
 
       switch (template.scope()) {
         case POINT_AGGREGATE:
-          return new DrillAggFuncHolder(functionAttributes, initializer);
+          return outputField.isComplexWriter() ?
+              new DrillComplexWriterAggFuncHolder(functionAttributes, initializer) :
+              new DrillAggFuncHolder(functionAttributes, initializer);
         case SIMPLE:
           return outputField.isComplexWriter() ?
               new DrillComplexWriterFuncHolder(functionAttributes, initializer) :
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionLookupContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionLookupContext.java
index 944041e805..872c13780d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionLookupContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionLookupContext.java
@@ -41,7 +41,7 @@
    * implementations found on classpath.
    *
    * @param functionCall - Specifies function name and type of arguments
-   * @return
+   * @return AbstractFuncHolder
    */
   public AbstractFuncHolder findNonDrillFunction(FunctionCall functionCall);
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/ModifiedUnparseVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/ModifiedUnparseVisitor.java
index 58e3c3a1f7..fe8349057e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/ModifiedUnparseVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/ModifiedUnparseVisitor.java
@@ -43,7 +43,7 @@ public static void main(String[] args) throws Exception {
   }
 
   /**
-   * Unparse the given {@link Java.CompilationUnit} to the given {@link Writer}.
+   * Unparse the given {@link org.codehaus.janino.Java.CompilationUnit} to the given {@link java.io.Writer}.
    */
   public static void unparse(Java.CompilationUnit cu, Writer w) {
     UnparseVisitor.unparse(cu, w);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/ByteSubstring.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/ByteSubstring.java
index bcbfeb5a6e..e1d02584bf 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/ByteSubstring.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/ByteSubstring.java
@@ -42,7 +42,8 @@
  */
 @FunctionTemplate(names = {"bytesubstring", "byte_substr"},
                   scope = FunctionTemplate.FunctionScope.SIMPLE,
-                  nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+                  nulls = FunctionTemplate.NullHandling.NULL_IF_NULL,
+                  outputWidthCalculatorType = FunctionTemplate.OutputWidthCalculatorType.CUSTOM_CLONE_DEFAULT)
 public class ByteSubstring implements DrillSimpleFunc {
 
   @Param VarBinaryHolder in;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/CastVarCharVar16Char.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/CastVarCharVar16Char.java
index edb929128f..1a929af341 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/CastVarCharVar16Char.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/CastVarCharVar16Char.java
@@ -34,7 +34,8 @@
 import org.apache.drill.exec.expr.holders.VarCharHolder;
 
 @SuppressWarnings("unused")
-@FunctionTemplate(names = {"castVAR16CHAR", "to_var16char", "to_string"}, scope = FunctionTemplate.FunctionScope.SIMPLE, nulls= NullHandling.NULL_IF_NULL)
+@FunctionTemplate(names = {"castVAR16CHAR", "to_var16char", "to_string"}, scope = FunctionTemplate.FunctionScope.SIMPLE, nulls= NullHandling.NULL_IF_NULL,
+                  outputWidthCalculatorType = FunctionTemplate.OutputWidthCalculatorType.CUSTOM_CLONE_DEFAULT)
 public class CastVarCharVar16Char implements DrillSimpleFunc {
   @Param VarCharHolder in;
   @Param BigIntHolder length;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/CharSubstring.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/CharSubstring.java
index ee6baff6d4..5c47692c0c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/CharSubstring.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/CharSubstring.java
@@ -46,7 +46,8 @@
  */
 @FunctionTemplate(names = {"charsubstring", "substring2", "substr2"},
                   scope = FunctionTemplate.FunctionScope.SIMPLE,
-                  nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+                  nulls = FunctionTemplate.NullHandling.NULL_IF_NULL,
+                  outputWidthCalculatorType = FunctionTemplate.OutputWidthCalculatorType.CLONE)
 public class CharSubstring implements DrillSimpleFunc {
 
   @Param VarCharHolder string;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/ContextFunctions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/ContextFunctions.java
index e3241f2840..590ab7d2d5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/ContextFunctions.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/ContextFunctions.java
@@ -24,6 +24,7 @@
 import org.apache.drill.exec.expr.annotations.Workspace;
 import org.apache.drill.exec.expr.holders.VarCharHolder;
 import org.apache.drill.exec.ops.ContextInformation;
+import org.apache.drill.exec.physical.impl.project.OutputSizeEstimateConstants;
 
 import javax.inject.Inject;
 
@@ -32,7 +33,8 @@
   /**
    * Implement "user", "session_user" or "system_user" function. Returns the username of the user connected to Drillbit.
    */
-  @FunctionTemplate(names = {"user", "session_user", "system_user"}, scope = FunctionTemplate.FunctionScope.SIMPLE, isNiladic = true)
+  @FunctionTemplate(names = {"user", "session_user", "system_user"}, scope = FunctionTemplate.FunctionScope.SIMPLE,
+          isNiladic = true, outputSizeEstimate = OutputSizeEstimateConstants.USER_NAME_LENGTH)
   public static class User implements DrillSimpleFunc {
     @Output VarCharHolder out;
     @Inject ContextInformation contextInfo;
@@ -58,7 +60,8 @@ public void eval() {
   /**
    * Implement "current_schema" function. Returns the default schema in current session.
    */
-  @FunctionTemplate(name = "current_schema", scope = FunctionTemplate.FunctionScope.SIMPLE, isNiladic = true)
+  @FunctionTemplate(name = "current_schema", scope = FunctionTemplate.FunctionScope.SIMPLE, isNiladic = true,
+                    outputSizeEstimate = OutputSizeEstimateConstants.SCHEMA_LENGTH)
   public static class CurrentSchema implements DrillSimpleFunc {
     @Output VarCharHolder out;
     @Inject ContextInformation contextInfo;
@@ -84,7 +87,8 @@ public void eval() {
   /**
    * Implement "session_id" function. Returns the unique id of the current session.
    */
-  @FunctionTemplate(name = "session_id", scope = FunctionTemplate.FunctionScope.SIMPLE, isNiladic = true)
+  @FunctionTemplate(name = "session_id", scope = FunctionTemplate.FunctionScope.SIMPLE, isNiladic = true,
+                    outputSizeEstimate = OutputSizeEstimateConstants.USER_ID_LENGTH)
   public static class SessionId implements DrillSimpleFunc {
     @Output VarCharHolder out;
     @Inject ContextInformation contextInfo;
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 07accf12d5..afd5f5d29c 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
@@ -36,6 +36,7 @@
 import org.apache.drill.exec.expr.holders.TimeStampHolder;
 import org.apache.drill.exec.expr.holders.VarCharHolder;
 import org.apache.drill.exec.ops.ContextInformation;
+import org.apache.drill.exec.physical.impl.project.OutputSizeEstimateConstants;
 
 import io.netty.buffer.DrillBuf;
 
@@ -269,7 +270,8 @@ public void eval() {
 
     }
 
-    @FunctionTemplate(name = "timeofday", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL, isRandom = true)
+    @FunctionTemplate(name = "timeofday", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL, isRandom = true,
+                      outputSizeEstimate = OutputSizeEstimateConstants.DATE_TIME_LENGTH)
     public static class TimeOfDay implements DrillSimpleFunc {
         @Inject DrillBuf buffer;
         @Output VarCharHolder out;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Mappify.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Mappify.java
index 703d62e02e..3db9f5ac39 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Mappify.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Mappify.java
@@ -60,7 +60,7 @@ public void setup() {
     }
 
     public void eval() {
-      buffer = org.apache.drill.exec.expr.fn.impl.MappifyUtility.mappify(reader, writer, buffer);
+      buffer = org.apache.drill.exec.expr.fn.impl.MappifyUtility.mappify(reader, writer, buffer, "Mappify/kvgen");
     }
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MappifyUtility.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MappifyUtility.java
index 3745fe2b34..b3fca2bb64 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MappifyUtility.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MappifyUtility.java
@@ -37,7 +37,7 @@
   public static final String fieldKey = "key";
   public static final String fieldValue = "value";
 
-  public static DrillBuf mappify(FieldReader reader, BaseWriter.ComplexWriter writer, DrillBuf buffer) {
+  public static DrillBuf mappify(FieldReader reader, BaseWriter.ComplexWriter writer, DrillBuf buffer, String caller) {
     // Currently we expect single map as input
     if (DataMode.REPEATED == reader.getType().getMode() || !(reader.getType().getMinorType() == TypeProtos.MinorType.MAP)) {
       throw new DrillRuntimeException("kvgen function only supports Simple maps as input");
@@ -72,7 +72,7 @@ public static DrillBuf mappify(FieldReader reader, BaseWriter.ComplexWriter writ
       mapWriter.varChar(fieldKey).write(vh);
 
       // Write the value to the map
-      MapUtility.writeToMapFromReader(fieldReader, mapWriter);
+      MapUtility.writeToMapFromReader(fieldReader, mapWriter, caller);
 
       mapWriter.end();
     }
@@ -80,5 +80,35 @@ public static DrillBuf mappify(FieldReader reader, BaseWriter.ComplexWriter writ
 
     return buffer;
   }
+
+  public static void createRepeatedMapOrList(FieldReader reader, BaseWriter.ComplexWriter writer, String caller) {
+    if (DataMode.REPEATED != reader.getType().getMode()) {
+      throw new DrillRuntimeException("Do not invoke createRepeatedMapOrList() unless MINOR mode is REPEATED");
+    }
+    BaseWriter.ListWriter listWriter = writer.rootAsList();
+    MapUtility.writeToListFromReader(reader, listWriter, caller);
+  }
+
+  public static void createMap(FieldReader reader, BaseWriter.ComplexWriter writer, String caller) {
+    if (DataMode.REPEATED == reader.getType().getMode()) {
+      throw new DrillRuntimeException("Do not invoke createMap() with REPEATED MINOR mode");
+    }
+    if (reader.getType().getMinorType() == TypeProtos.MinorType.MAP) {
+      BaseWriter.MapWriter mapWriter = writer.rootAsMap();
+      // Iterate over the fields in the map
+      Iterator<String> fieldIterator = reader.iterator();
+      while (fieldIterator.hasNext()) {
+        String field = fieldIterator.next();
+        FieldReader fieldReader = reader.reader(field);
+        // Write the value to the map
+        MapUtility.writeToMapFromReader(fieldReader, mapWriter, field, caller);
+      }
+    }
+  }
+
+  public static void createList(FieldReader reader, BaseWriter.ComplexWriter writer, String caller) {
+    BaseWriter.ListWriter listWriter = writer.rootAsList();
+    MapUtility.writeToListFromReader(reader, listWriter, caller);
+  }
 }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/PhoneticFunctions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/PhoneticFunctions.java
new file mode 100644
index 0000000000..ee26bd3ec8
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/PhoneticFunctions.java
@@ -0,0 +1,407 @@
+/*
+ * 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.fn.impl;
+
+import io.netty.buffer.DrillBuf;
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.VarCharHolder;
+
+import javax.inject.Inject;
+
+public class PhoneticFunctions {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PhoneticFunctions.class);
+
+  private PhoneticFunctions() {
+  }
+
+  /**
+   * The Caverphone function is a phonetic matching function.   This is an algorithm created by the Caversham Project at the University of Otago. It implements the Caverphone 1.0 algorithm.
+   * <p>
+   * <p>
+   * Usage:  SELECT caverphone1( string ) FROM...
+   */
+
+  @FunctionTemplate(name = "caverphone1", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class Caverphone1Function implements DrillSimpleFunc {
+
+    @Param
+    VarCharHolder rawInput;
+
+    @Output
+    VarCharHolder out;
+
+    @Inject
+    DrillBuf buffer;
+
+    @Override
+    public void setup() {
+    }
+
+    @Override
+    public void eval() {
+
+      String input = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(rawInput.start, rawInput.end, rawInput.buffer);
+      String outputString = new org.apache.commons.codec.language.Caverphone1().encode(input);
+
+      out.buffer = buffer;
+      out.start = 0;
+      out.end = outputString.getBytes().length;
+      buffer.setBytes(0, outputString.getBytes());
+    }
+
+  }
+
+  /**
+   * The Caverphone function is a phonetic matching function.   This is an algorithm created by the Caversham Project at the University of Otago. It implements the Caverphone 2.0 algorithm.
+   * <p>
+   * Usage: SELECT caverphone2( string ) FROM...
+   */
+
+  @FunctionTemplate(name = "caverphone2", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class Caverphone2Function implements DrillSimpleFunc {
+
+    @Param
+    VarCharHolder rawInput;
+
+    @Output
+    VarCharHolder out;
+
+    @Inject
+    DrillBuf buffer;
+
+    @Override
+    public void setup() {
+    }
+
+    @Override
+    public void eval() {
+
+      String input = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(rawInput.start, rawInput.end, rawInput.buffer);
+      String outputString = new org.apache.commons.codec.language.Caverphone2().encode(input);
+
+      out.buffer = buffer;
+      out.start = 0;
+      out.end = outputString.getBytes().length;
+      buffer.setBytes(0, outputString.getBytes());
+    }
+
+  }
+
+  /**
+   * Encodes a string into a Cologne Phonetic value.
+   * Implements the Kölner Phonetik (Cologne Phonetic) algorithm issued by Hans Joachim Postel in 1969.
+   * <p>
+   * The Kölner Phonetik is a phonetic algorithm which is optimized for the German language.
+   * It is related to the well-known soundex algorithm.
+   * <p>
+   * Usage:  SELECT cologne_phonetic( string ) FROM...
+   */
+
+  @FunctionTemplate(name = "cologne_phonetic", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class ColognePhoneticFunction implements DrillSimpleFunc {
+
+    @Param
+    VarCharHolder rawInput;
+
+    @Output
+    VarCharHolder out;
+
+    @Inject
+    DrillBuf buffer;
+
+    @Override
+    public void setup() {
+    }
+
+    @Override
+    public void eval() {
+
+      String input = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(rawInput.start, rawInput.end, rawInput.buffer);
+      String outputString = new org.apache.commons.codec.language.ColognePhonetic().encode(input);
+
+      out.buffer = buffer;
+      out.start = 0;
+      out.end = outputString.getBytes().length;
+      buffer.setBytes(0, outputString.getBytes());
+    }
+
+  }
+
+  /**
+   * Encodes a string into a Daitch-Mokotoff Soundex value.
+   * The Daitch-Mokotoff Soundex algorithm is a refinement of the Russel and American Soundex algorithms,
+   * yielding greater accuracy in matching especially Slavish and Yiddish surnames with similar pronunciation
+   * but differences in spelling.
+   * <p>
+   * The main differences compared to the other soundex variants are:
+   * coded names are 6 digits long
+   * the initial character of the name is coded
+   * rules to encoded multi-character n-grams
+   * multiple possible encodings for the same name (branching)
+   * <p>
+   * Usage:  SELECT dm_soundex( string ) FROM...
+   */
+
+  @FunctionTemplate(name = "dm_soundex", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class DaitchMokotoffFunction implements DrillSimpleFunc {
+
+    @Param
+    VarCharHolder rawInput;
+
+    @Output
+    VarCharHolder out;
+
+    @Inject
+    DrillBuf buffer;
+
+    @Override
+    public void setup() {
+    }
+
+    @Override
+    public void eval() {
+
+      String input = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(rawInput.start, rawInput.end, rawInput.buffer);
+      String outputString = new org.apache.commons.codec.language.DaitchMokotoffSoundex().encode(input);
+
+      out.buffer = buffer;
+      out.start = 0;
+      out.end = outputString.getBytes().length;
+      buffer.setBytes(0, outputString.getBytes());
+    }
+
+  }
+
+  /**
+   * Match Rating Approach Phonetic Algorithm Developed by Western Airlines in 1977.
+   * Usage:  SELECT match_rating_encoder( string ) FROM...
+   */
+
+  @FunctionTemplate(name = "match_rating_encoder", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class MatchRatingFunction implements DrillSimpleFunc {
+
+    @Param
+    VarCharHolder rawInput;
+
+    @Output
+    VarCharHolder out;
+
+    @Inject
+    DrillBuf buffer;
+
+    @Override
+    public void setup() {
+    }
+
+    @Override
+    public void eval() {
+
+      String input = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(rawInput.start, rawInput.end, rawInput.buffer);
+      String outputString = new org.apache.commons.codec.language.MatchRatingApproachEncoder().encode(input);
+
+      out.buffer = buffer;
+      out.start = 0;
+      out.end = outputString.getBytes().length;
+      buffer.setBytes(0, outputString.getBytes());
+    }
+
+  }
+
+  /**
+   * The New York State Identification and Intelligence System Phonetic Code, commonly known as NYSIIS, is a phonetic algorithm devised in 1970 as part of the New York State Identification and Intelligence System (now a part of the New York State Division of Criminal Justice Services). It features an accuracy increase of 2.7% over the traditional Soundex algorithm.
+   * Encodes a string into a NYSIIS value. NYSIIS is an encoding used to relate similar names, but can also be used as a general purpose scheme to find word with similar phonemes.
+   * <p>
+   * Usage: SELECT nysiis(string) FROM...
+   */
+
+  @FunctionTemplate(name = "nysiis", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class NYSIISFunction implements DrillSimpleFunc {
+
+    @Param
+    VarCharHolder rawInput;
+
+    @Output
+    VarCharHolder out;
+
+    @Inject
+    DrillBuf buffer;
+
+    @Override
+    public void setup() {
+    }
+
+    @Override
+    public void eval() {
+
+      String input = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(rawInput.start, rawInput.end, rawInput.buffer);
+      String outputString = new org.apache.commons.codec.language.Nysiis().encode(input);
+
+      out.buffer = buffer;
+      out.start = 0;
+      out.end = outputString.getBytes().length;
+      buffer.setBytes(0, outputString.getBytes());
+    }
+  }
+
+  /**
+   * Encodes a string into a Refined Soundex value. Soundex is an encoding used to relate similar names, but can also be used as a general purpose scheme to find word with similar phonemes.
+   * <p>
+   * Usage:  SELECT refined_soundex( string ) FROM...
+   */
+
+  @FunctionTemplate(name = "refined_soundex", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class RefinedSoundexFunction implements DrillSimpleFunc {
+
+    @Param
+    VarCharHolder rawInput;
+
+    @Output
+    VarCharHolder out;
+
+    @Inject
+    DrillBuf buffer;
+
+    @Override
+    public void setup() {
+    }
+
+    @Override
+    public void eval() {
+
+      String input = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(rawInput.start, rawInput.end, rawInput.buffer);
+      String outputString = new org.apache.commons.codec.language.RefinedSoundex().encode(input);
+
+      out.buffer = buffer;
+      out.start = 0;
+      out.end = outputString.getBytes().length;
+      buffer.setBytes(0, outputString.getBytes());
+    }
+
+  }
+
+  /**
+   * Encodes a string into a Soundex value. Soundex is an encoding used to relate similar names, but can also be used as a general purpose scheme to find word with similar phonemes.
+   * <p>
+   * Usage:  SELECT soundex( string ) FROM...
+   */
+
+  @FunctionTemplate(name = "soundex", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class SoundexFunction implements DrillSimpleFunc {
+
+    @Param
+    VarCharHolder rawInput;
+
+    @Output
+    VarCharHolder out;
+
+    @Inject
+    DrillBuf buffer;
+
+    @Override
+    public void setup() {
+    }
+
+    @Override
+    public void eval() {
+
+      String input = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(rawInput.start, rawInput.end, rawInput.buffer);
+      String outputString = new org.apache.commons.codec.language.Soundex().soundex(input);
+
+      out.buffer = buffer;
+      out.start = 0;
+      out.end = outputString.getBytes().length;
+      buffer.setBytes(0, outputString.getBytes());
+    }
+  }
+
+  /**
+   * Implements the Metaphone phonetic algorithm (https://en.wikipedia.org/wiki/Metaphone),
+   * and calculates a given string's Metaphone value.
+   * <p>
+   * Usage: SELECT metaphone( string ) FROM...
+   */
+
+  @FunctionTemplate(name = "metaphone", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class MetaphoneFunction implements DrillSimpleFunc {
+
+    @Param
+    VarCharHolder rawInput;
+
+    @Output
+    VarCharHolder out;
+
+    @Inject
+    DrillBuf buffer;
+
+    @Override
+    public void setup() {
+    }
+
+    @Override
+    public void eval() {
+
+      String input = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(rawInput.start, rawInput.end, rawInput.buffer);
+      String outputString = new org.apache.commons.codec.language.Metaphone().metaphone(input);
+
+      out.buffer = buffer;
+      out.start = 0;
+      out.end = outputString.getBytes().length;
+      buffer.setBytes(0, outputString.getBytes());
+    }
+
+  }
+
+  /**
+   * Implements the Double Metaphone phonetic algorithm (https://en.wikipedia.org/wiki/Metaphone),
+   * and calculates a given string's Double Metaphone value.
+   * <p>
+   * Usage: SELECT double_metaphone( string ) FROM...
+   */
+
+  @FunctionTemplate(name = "double_metaphone", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class DoubleMetaphoneFunction implements DrillSimpleFunc {
+
+    @Param
+    VarCharHolder rawInput;
+
+    @Output
+    VarCharHolder out;
+
+    @Inject
+    DrillBuf buffer;
+
+    @Override
+    public void setup() {
+    }
+
+    @Override
+    public void eval() {
+
+      String input = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(rawInput.start, rawInput.end, rawInput.buffer);
+      String outputString = new org.apache.commons.codec.language.DoubleMetaphone().doubleMetaphone(input);
+
+      out.buffer = buffer;
+      out.start = 0;
+      out.end = outputString.getBytes().length;
+      buffer.setBytes(0, outputString.getBytes());
+    }
+  }
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/SimpleCastFunctions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/SimpleCastFunctions.java
index 807fbb9181..196445ed5d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/SimpleCastFunctions.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/SimpleCastFunctions.java
@@ -54,7 +54,8 @@ public void eval() {
   @FunctionTemplate(name = "castVARCHAR",
       scope = FunctionTemplate.FunctionScope.SIMPLE,
       returnType = FunctionTemplate.ReturnType.STRING_CAST,
-      nulls = NullHandling.NULL_IF_NULL)
+      nulls = NullHandling.NULL_IF_NULL,
+      outputWidthCalculatorType = FunctionTemplate.OutputWidthCalculatorType.CUSTOM_CLONE_DEFAULT)
   public static class CastBooleanVarChar implements DrillSimpleFunc {
 
     @Param BitHolder in;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringDistanceFunctions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringDistanceFunctions.java
new file mode 100644
index 0000000000..0b02769445
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringDistanceFunctions.java
@@ -0,0 +1,329 @@
+/*
+ * 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.fn.impl;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.expr.holders.Float8Holder;
+import org.apache.drill.exec.expr.holders.VarCharHolder;
+
+public class StringDistanceFunctions {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StringDistanceFunctions.class);
+
+  private StringDistanceFunctions() {
+  }
+
+  /**
+   * This function calculates the cosine distance between two strings.
+   * Usage:  SELECT cosine_distance( string1, string2 ) AS cosine_distance FROM...
+   */
+
+  @FunctionTemplate(name = "cosine_distance", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class CosineDistanceFunction implements DrillSimpleFunc {
+
+    @Param
+    VarCharHolder rawInput1;
+
+    @Param
+    VarCharHolder rawInput2;
+
+    @Workspace
+    org.apache.commons.text.similarity.CosineDistance d;
+
+    @Output
+    Float8Holder out;
+
+    @Override
+    public void setup() {
+      d = new org.apache.commons.text.similarity.CosineDistance();
+    }
+
+    @Override
+    public void eval() {
+
+      String input1 = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(rawInput1.start, rawInput1.end, rawInput1.buffer);
+      String input2 = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(rawInput2.start, rawInput2.end, rawInput2.buffer);
+
+
+      double result = d.apply(input1, input2);
+      out.value = result;
+    }
+  }
+
+  /**
+   * This function calculates the cosine distance between two strings.
+   * A matching algorithm that is similar to the searching algorithms implemented in editors such
+   * as Sublime Text, TextMate, Atom and others.
+   * <p>
+   * One point is given for every matched character. Subsequent matches yield two bonus points. A higher score
+   * indicates a higher similarity.
+   * <p>
+   * <p>
+   * Usage:  SELECT fuzzy_score( string1, string2 ) AS fuzzy_score FROM...
+   */
+
+  @FunctionTemplate(name = "fuzzy_score", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class FuzzyScoreFunction implements DrillSimpleFunc {
+
+    @Param
+    VarCharHolder rawInput1;
+
+    @Param
+    VarCharHolder rawInput2;
+
+    @Output
+    Float8Holder out;
+
+    @Workspace
+    org.apache.commons.text.similarity.FuzzyScore d;
+
+    @Override
+    public void setup() {
+      d = new org.apache.commons.text.similarity.FuzzyScore(java.util.Locale.ENGLISH);
+    }
+
+    @Override
+    public void eval() {
+
+      String input1 = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(rawInput1.start, rawInput1.end, rawInput1.buffer);
+      String input2 = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(rawInput2.start, rawInput2.end, rawInput2.buffer);
+
+      double result = d.fuzzyScore(input1, input2);
+      out.value = result;
+    }
+  }
+
+  /**
+   * The hamming distance between two strings of equal length is the number of
+   * positions at which the corresponding symbols are different.
+   * <p>
+   * For further explanation about the Hamming Distance, take a look at its
+   * Wikipedia page at http://en.wikipedia.org/wiki/Hamming_distance.
+   * <p>
+   * Usage:  SELECT hamming_distance( string1, string2 ) FROM...
+   */
+
+
+  @FunctionTemplate(name = "hamming_distance", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class HammingDistanceFunction implements DrillSimpleFunc {
+
+    @Param
+    VarCharHolder rawInput1;
+
+    @Param
+    VarCharHolder rawInput2;
+
+    @Output
+    Float8Holder out;
+
+    @Workspace
+    org.apache.commons.text.similarity.HammingDistance d;
+
+    @Override
+    public void setup() {
+      d = new org.apache.commons.text.similarity.HammingDistance();
+    }
+
+    @Override
+    public void eval() {
+
+      String input1 = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(rawInput1.start, rawInput1.end, rawInput1.buffer);
+      String input2 = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(rawInput2.start, rawInput2.end, rawInput2.buffer);
+
+      double result = d.apply(input1, input2);
+      out.value = result;
+    }
+  }
+
+
+  /**
+   * Measures the Jaccard distance of two sets of character sequence. Jaccard
+   * distance is the dissimilarity between two sets. It is the complementary of
+   * Jaccard similarity.
+   * <p>
+   * For further explanation about Jaccard Distance, refer
+   * https://en.wikipedia.org/wiki/Jaccard_index
+   * <p>
+   * Usage:  SELECT jaccard_distance( string1, string2 ) FROM ...
+   */
+
+
+  @FunctionTemplate(name = "jaccard_distance", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class JaccardDistanceFunction implements DrillSimpleFunc {
+
+    @Param
+    VarCharHolder rawInput1;
+
+    @Param
+    VarCharHolder rawInput2;
+
+    @Output
+    Float8Holder out;
+
+    @Workspace
+    org.apache.commons.text.similarity.JaccardDistance d;
+
+    @Override
+    public void setup() {
+      d = new org.apache.commons.text.similarity.JaccardDistance();
+    }
+
+    @Override
+    public void eval() {
+
+      String input1 = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(rawInput1.start, rawInput1.end, rawInput1.buffer);
+      String input2 = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(rawInput2.start, rawInput2.end, rawInput2.buffer);
+
+      double result = d.apply(input1, input2);
+      out.value = result;
+    }
+  }
+
+  /**
+   * A similarity algorithm indicating the percentage of matched characters between two character sequences.
+   * <p>
+   * The Jaro measure is the weighted sum of percentage of matched characters
+   * from each file and transposed characters. Winkler increased this measure
+   * for matching initial characters.
+   * <p>
+   * This implementation is based on the Jaro Winkler similarity algorithm
+   * from https://en.wikipedia.org/wiki/Jaro–Winkler_distance
+   * <p>
+   * Usage: SELECT jaro_distance( string1, string2 ) FROM...
+   */
+
+  @FunctionTemplate(name = "jaro_distance", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class JaroDistanceFunction implements DrillSimpleFunc {
+
+    @Param
+    VarCharHolder rawInput1;
+
+    @Param
+    VarCharHolder rawInput2;
+
+    @Output
+    Float8Holder out;
+
+    @Workspace
+    org.apache.commons.text.similarity.JaroWinklerDistance d;
+
+    @Override
+    public void setup() {
+      d = new org.apache.commons.text.similarity.JaroWinklerDistance();
+    }
+
+    @Override
+    public void eval() {
+
+      String input1 = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(rawInput1.start, rawInput1.end, rawInput1.buffer);
+      String input2 = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(rawInput2.start, rawInput2.end, rawInput2.buffer);
+
+      double result = d.apply(input1, input2);
+      out.value = result;
+    }
+  }
+
+  /**
+   * An algorithm for measuring the difference between two character sequences.
+   * <p>
+   * This is the number of changes needed to change one sequence into another,
+   * where each change is a single character modification (deletion, insertion
+   * or substitution).
+   * <p>
+   * Usage: SELECT levenshtein_distance( string1, string2 ) FROM...
+   */
+
+  @FunctionTemplate(name = "levenshtein_distance", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class LevenstheinDistanceFunction implements DrillSimpleFunc {
+
+    @Param
+    VarCharHolder rawInput1;
+
+    @Param
+    VarCharHolder rawInput2;
+
+    @Output
+    Float8Holder out;
+
+    @Workspace
+    org.apache.commons.text.similarity.LevenshteinDistance d;
+
+    @Override
+    public void setup() {
+      d = new org.apache.commons.text.similarity.LevenshteinDistance();
+    }
+
+    @Override
+    public void eval() {
+
+      String input1 = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(rawInput1.start, rawInput1.end, rawInput1.buffer);
+      String input2 = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(rawInput2.start, rawInput2.end, rawInput2.buffer);
+
+      double result = d.apply(input1, input2);
+      out.value = result;
+    }
+  }
+
+  /**
+   * The Longest common subsequence algorithm returns the length of the longest subsequence that two strings have in common.
+   * Two strings that are entirely different, return a value of 0, and two strings that return a value of the
+   * commonly shared length implies that the strings are completely the same in value and position.
+   * Note: Generally this algorithm is fairly inefficient, as for length m, n of the input
+   * CharSequence's left and right respectively, the runtime of the algorithm is O(m*n).
+   * <p>
+   * This implementation is based on the Longest Commons Substring algorithm from https://en.wikipedia.org/wiki/Longest_common_subsequence_problem.
+   * <p>
+   * Usage:  SELECT longest_common_substring_distance( string1, string2 ) FROM...
+   */
+
+  @FunctionTemplate(name = "longest_common_substring_distance", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class LongestCommonSubstringDistanceFunction implements DrillSimpleFunc {
+
+    @Param
+    VarCharHolder rawInput1;
+
+    @Param
+    VarCharHolder rawInput2;
+
+    @Output
+    Float8Holder out;
+
+    @Workspace
+    org.apache.commons.text.similarity.LongestCommonSubsequenceDistance d;
+
+    @Override
+    public void setup() {
+      d = new org.apache.commons.text.similarity.LongestCommonSubsequenceDistance();
+    }
+
+    @Override
+    public void eval() {
+
+      String input1 = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(rawInput1.start, rawInput1.end, rawInput1.buffer);
+      String input2 = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(rawInput2.start, rawInput2.end, rawInput2.buffer);
+
+      double result = d.apply(input1, input2);
+      out.value = result;
+    }
+  }
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java
index ad3f379160..43098486cc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java
@@ -20,16 +20,12 @@
 
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.DrillBuf;
-
-import java.nio.charset.Charset;
-
-import javax.inject.Inject;
-
 import org.apache.drill.exec.expr.DrillSimpleFunc;
 import org.apache.drill.exec.expr.annotations.FunctionTemplate;
 import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
 import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
 import org.apache.drill.exec.expr.annotations.FunctionTemplate.ReturnType;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.OutputWidthCalculatorType;
 import org.apache.drill.exec.expr.annotations.Output;
 import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.annotations.Workspace;
@@ -39,6 +35,10 @@
 import org.apache.drill.exec.expr.holders.NullableVarCharHolder;
 import org.apache.drill.exec.expr.holders.VarBinaryHolder;
 import org.apache.drill.exec.expr.holders.VarCharHolder;
+import org.apache.drill.exec.physical.impl.project.OutputSizeEstimateConstants;
+
+import javax.inject.Inject;
+import java.nio.charset.Charset;
 
 public class StringFunctions{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StringFunctions.class);
@@ -211,7 +211,8 @@ public void eval() {
   /*
    * Replace all substring that match the regular expression with replacement.
    */
-  @FunctionTemplate(name = "regexp_replace", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  @FunctionTemplate(name = "regexp_replace", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                    outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_CLONE_DEFAULT)
   public static class RegexpReplace implements DrillSimpleFunc {
 
     @Param VarCharHolder input;
@@ -383,7 +384,8 @@ public void eval() {
   }
 
 
-  @FunctionTemplate(name = "split_part", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  @FunctionTemplate(name = "split_part", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                    outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_FIXED_WIDTH_DEFUALT)
   public static class SplitPart implements DrillSimpleFunc {
     @Param  VarCharHolder str;
     @Param  VarCharHolder splitter;
@@ -477,7 +479,8 @@ public void eval() {
   @FunctionTemplate(name = "lower",
       scope = FunctionScope.SIMPLE,
       returnType = ReturnType.SAME_IN_OUT_LENGTH,
-      nulls = NullHandling.NULL_IF_NULL)
+      nulls = NullHandling.NULL_IF_NULL,
+      outputWidthCalculatorType = OutputWidthCalculatorType.CLONE)
   public static class LowerCase implements DrillSimpleFunc {
     @Param VarCharHolder input;
     @Output VarCharHolder out;
@@ -506,6 +509,7 @@ public void eval() {
   @FunctionTemplate(name = "upper",
       scope = FunctionScope.SIMPLE,
       returnType = ReturnType.SAME_IN_OUT_LENGTH,
+      outputWidthCalculatorType = OutputWidthCalculatorType.CLONE,
       nulls = NullHandling.NULL_IF_NULL)
   public static class UpperCase implements DrillSimpleFunc {
 
@@ -534,7 +538,8 @@ public void eval() {
   // Follow Postgre.
   //  -- Valid "offset": [1, string_length],
   //  -- Valid "length": [1, up to string_length - offset + 1], if length > string_length - offset +1, get the substr up to the string_lengt.
-  @FunctionTemplate(names = {"substring", "substr"}, scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  @FunctionTemplate(names = {"substring", "substr"}, scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                    outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_CLONE_DEFAULT)
   public static class Substring implements DrillSimpleFunc {
     @Param VarCharHolder string;
     @Param BigIntHolder offset;
@@ -572,7 +577,8 @@ public void eval() {
     }
   }
 
-  @FunctionTemplate(names = {"substring", "substr"}, scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  @FunctionTemplate(names = {"substring", "substr"}, scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                    outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_CLONE_DEFAULT)
   public static class SubstringOffset implements DrillSimpleFunc {
     @Param VarCharHolder string;
     @Param BigIntHolder offset;
@@ -605,7 +611,8 @@ public void eval() {
     }
   }
 
-  @FunctionTemplate(names = {"substring", "substr" }, scope = FunctionScope.SIMPLE, nulls = NullHandling.INTERNAL)
+  @FunctionTemplate(names = {"substring", "substr" }, scope = FunctionScope.SIMPLE, nulls = NullHandling.INTERNAL,
+                    outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_CLONE_DEFAULT)
   public static class SubstringRegex implements DrillSimpleFunc {
     @Param VarCharHolder input;
     @Param(constant=true) VarCharHolder pattern;
@@ -637,7 +644,8 @@ public void eval() {
     }
   }
 
-  @FunctionTemplate(names = {"substring", "substr" }, scope = FunctionScope.SIMPLE, nulls = NullHandling.INTERNAL)
+  @FunctionTemplate(names = {"substring", "substr" }, scope = FunctionScope.SIMPLE, nulls = NullHandling.INTERNAL,
+                    outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_FIXED_WIDTH_DEFUALT)
   public static class SubstringRegexNullable implements DrillSimpleFunc {
     @Param NullableVarCharHolder input;
     @Param(constant=true) VarCharHolder pattern;
@@ -679,7 +687,8 @@ public void eval() {
   // If length > total charcounts, return the whole string.
   // If length = 0, return empty
   // If length < 0, and |length| > total charcounts, return empty.
-  @FunctionTemplate(name = "left", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  @FunctionTemplate(name = "left", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                    outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_FIXED_WIDTH_DEFUALT)
   public static class Left implements DrillSimpleFunc {
     @Param VarCharHolder string;
     @Param BigIntHolder length;
@@ -716,7 +725,8 @@ public void eval() {
   }
 
   //Return last 'length' characters in the string. When 'length' is negative, return all but first |length| characters.
-  @FunctionTemplate(name = "right", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  @FunctionTemplate(name = "right", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                    outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_FIXED_WIDTH_DEFUALT)
   public static class Right implements DrillSimpleFunc {
     @Param VarCharHolder string;
     @Param BigIntHolder length;
@@ -763,7 +773,8 @@ public void eval() {
   @FunctionTemplate(name = "initcap",
       scope = FunctionScope.SIMPLE,
       returnType = ReturnType.SAME_IN_OUT_LENGTH,
-      nulls = NullHandling.NULL_IF_NULL)
+      nulls = NullHandling.NULL_IF_NULL,
+      outputWidthCalculatorType = OutputWidthCalculatorType.CLONE)
   public static class InitCap implements DrillSimpleFunc {
     @Param VarCharHolder input;
     @Output VarCharHolder out;
@@ -784,7 +795,8 @@ public void eval() {
   }
 
   //Replace all occurrences in 'text' of substring 'from' with substring 'to'
-  @FunctionTemplate(name = "replace", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  @FunctionTemplate(name = "replace", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                    outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_CLONE_DEFAULT)
   public static class Replace implements DrillSimpleFunc {
     @Param  VarCharHolder text;
     @Param  VarCharHolder from;
@@ -851,7 +863,8 @@ public void eval() {
   @FunctionTemplate(name = "lpad",
       scope = FunctionScope.SIMPLE,
       returnType = ReturnType.PAD,
-      nulls = NullHandling.NULL_IF_NULL)
+      nulls = NullHandling.NULL_IF_NULL,
+      outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_FIXED_WIDTH_DEFUALT)
   public static class Lpad implements DrillSimpleFunc {
     @Param  VarCharHolder text;
     @Param  BigIntHolder length;
@@ -929,7 +942,8 @@ public void eval() {
   @FunctionTemplate(name = "lpad",
       scope = FunctionScope.SIMPLE,
       returnType = ReturnType.PAD,
-      nulls = NullHandling.NULL_IF_NULL)
+      nulls = NullHandling.NULL_IF_NULL,
+      outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_FIXED_WIDTH_DEFUALT)
   public static class LpadTwoArg implements DrillSimpleFunc {
     @Param  VarCharHolder text;
     @Param  BigIntHolder length;
@@ -991,7 +1005,8 @@ public void eval() {
   @FunctionTemplate(name = "rpad",
       scope = FunctionScope.SIMPLE,
       returnType = ReturnType.PAD,
-      nulls = NullHandling.NULL_IF_NULL)
+      nulls = NullHandling.NULL_IF_NULL,
+      outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_FIXED_WIDTH_DEFUALT)
   public static class Rpad implements DrillSimpleFunc {
     @Param  VarCharHolder text;
     @Param  BigIntHolder length;
@@ -1072,7 +1087,8 @@ public void eval() {
   @FunctionTemplate(name = "rpad",
       scope = FunctionScope.SIMPLE,
       returnType = ReturnType.PAD,
-      nulls = NullHandling.NULL_IF_NULL)
+      nulls = NullHandling.NULL_IF_NULL,
+      outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_FIXED_WIDTH_DEFUALT)
   public static class RpadTwoArg implements DrillSimpleFunc {
     @Param  VarCharHolder text;
     @Param  BigIntHolder length;
@@ -1133,7 +1149,8 @@ public void eval() {
   /**
    * Remove the longest string containing only characters from "from"  from the start of "text"
    */
-  @FunctionTemplate(name = "ltrim", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  @FunctionTemplate(name = "ltrim", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                    outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_CLONE_DEFAULT)
   public static class Ltrim implements DrillSimpleFunc {
     @Param  VarCharHolder text;
     @Param  VarCharHolder from;
@@ -1166,7 +1183,8 @@ public void eval() {
   /**
    * Remove the longest string containing only character " " from the start of "text"
    */
-  @FunctionTemplate(name = "ltrim", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  @FunctionTemplate(name = "ltrim", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                    outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_CLONE_DEFAULT)
   public static class LtrimOneArg implements DrillSimpleFunc {
     @Param  VarCharHolder text;
 
@@ -1196,7 +1214,8 @@ public void eval() {
   /**
    * Remove the longest string containing only characters from "from"  from the end of "text"
    */
-  @FunctionTemplate(name = "rtrim", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  @FunctionTemplate(name = "rtrim", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                    outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_CLONE_DEFAULT)
   public static class Rtrim implements DrillSimpleFunc {
     @Param  VarCharHolder text;
     @Param  VarCharHolder from;
@@ -1232,7 +1251,8 @@ public void eval() {
   /**
    * Remove the longest string containing only character " " from the end of "text"
    */
-  @FunctionTemplate(name = "rtrim", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  @FunctionTemplate(name = "rtrim", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                    outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_CLONE_DEFAULT)
   public static class RtrimOneArg implements DrillSimpleFunc {
     @Param  VarCharHolder text;
 
@@ -1265,7 +1285,8 @@ public void eval() {
   /**
    * Remove the longest string containing only characters from "from"  from the start of "text"
    */
-  @FunctionTemplate(name = "btrim", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  @FunctionTemplate(name = "btrim", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                    outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_CLONE_DEFAULT)
   public static class Btrim implements DrillSimpleFunc {
     @Param  VarCharHolder text;
     @Param  VarCharHolder from;
@@ -1312,7 +1333,8 @@ public void eval() {
   /**
    * Remove the longest string containing only character " " from the start of "text"
    */
-  @FunctionTemplate(name = "btrim", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  @FunctionTemplate(name = "btrim", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                    outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_CLONE_DEFAULT)
   public static class BtrimOneArg implements DrillSimpleFunc {
     @Param  VarCharHolder text;
 
@@ -1350,7 +1372,8 @@ public void eval() {
     } // end of eval
   }
 
-  @FunctionTemplate(name = "split", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  @FunctionTemplate(name = "split", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                    outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_FIXED_WIDTH_DEFUALT)
   public static class Split implements DrillSimpleFunc {
     @Param  VarCharHolder input;
     @Param  VarCharHolder delimiter;
@@ -1392,7 +1415,8 @@ public void eval() {
   @FunctionTemplate(name = "concatOperator",
       scope = FunctionScope.SIMPLE,
       returnType = ReturnType.CONCAT,
-      nulls = NullHandling.NULL_IF_NULL)
+      nulls = NullHandling.NULL_IF_NULL,
+      outputWidthCalculatorType = OutputWidthCalculatorType.CONCAT)
   public static class ConcatOperator implements DrillSimpleFunc {
     @Param  VarCharHolder left;
     @Param  VarCharHolder right;
@@ -1424,6 +1448,7 @@ public void eval() {
   @FunctionTemplate(name = "concat",
       scope = FunctionScope.SIMPLE,
       returnType = ReturnType.CONCAT,
+      outputWidthCalculatorType = OutputWidthCalculatorType.CONCAT,
       nulls = NullHandling.INTERNAL)
   public static class Concat implements DrillSimpleFunc {
     @Param  VarCharHolder left;
@@ -1454,7 +1479,8 @@ public void eval() {
   @FunctionTemplate(name = "concat",
       scope = FunctionScope.SIMPLE,
       returnType = ReturnType.CONCAT,
-      nulls = NullHandling.INTERNAL)
+      nulls = NullHandling.INTERNAL,
+      outputWidthCalculatorType = OutputWidthCalculatorType.CONCAT)
   public static class ConcatRightNullInput implements DrillSimpleFunc {
     @Param  VarCharHolder left;
     @Param  NullableVarCharHolder right;
@@ -1486,7 +1512,8 @@ public void eval() {
   @FunctionTemplate(name = "concat",
       scope = FunctionScope.SIMPLE,
       returnType = ReturnType.CONCAT,
-      nulls = NullHandling.INTERNAL)
+      nulls = NullHandling.INTERNAL,
+      outputWidthCalculatorType = OutputWidthCalculatorType.CONCAT)
   public static class ConcatLeftNullInput implements DrillSimpleFunc {
     @Param  NullableVarCharHolder left;
     @Param  VarCharHolder right;
@@ -1518,7 +1545,8 @@ public void eval() {
   @FunctionTemplate(name = "concat",
       scope = FunctionScope.SIMPLE,
       returnType = ReturnType.CONCAT,
-      nulls = NullHandling.INTERNAL)
+      nulls = NullHandling.INTERNAL,
+      outputWidthCalculatorType = OutputWidthCalculatorType.CONCAT)
   public static class ConcatBothNullInput implements DrillSimpleFunc {
     @Param  NullableVarCharHolder left;
     @Param  NullableVarCharHolder right;
@@ -1551,7 +1579,8 @@ public void eval() {
 
   // Converts a hex encoded string into a varbinary type.
   // "\xca\xfe\xba\xbe" => (byte[]) {(byte)0xca, (byte)0xfe, (byte)0xba, (byte)0xbe}
-  @FunctionTemplate(name = "binary_string", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  @FunctionTemplate(name = "binary_string", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                    outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_CLONE_DEFAULT)
   public static class BinaryString implements DrillSimpleFunc {
     @Param  VarCharHolder in;
     @Output VarBinaryHolder out;
@@ -1571,7 +1600,8 @@ public void eval() {
 
   // Converts a varbinary type into a hex encoded string.
   // (byte[]) {(byte)0xca, (byte)0xfe, (byte)0xba, (byte)0xbe}  => "\xca\xfe\xba\xbe"
-  @FunctionTemplate(name = "string_binary", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  @FunctionTemplate(name = "string_binary", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                    outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_CLONE_DEFAULT)
   public static class StringBinary implements DrillSimpleFunc {
     @Param  VarBinaryHolder in;
     @Output VarCharHolder   out;
@@ -1615,7 +1645,8 @@ public void eval() {
   /**
   * Returns the char corresponding to ASCII code input.
   */
-  @FunctionTemplate(name = "chr", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  @FunctionTemplate(name = "chr", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                    outputSizeEstimate = OutputSizeEstimateConstants.CHAR_LENGTH)
   public static class AsciiToChar implements DrillSimpleFunc {
     @Param  IntHolder in;
     @Output VarCharHolder out;
@@ -1638,7 +1669,8 @@ public void eval() {
   /**
   * Returns the input char sequences repeated nTimes.
   */
-  @FunctionTemplate(names = {"repeat", "repeatstr"}, scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  @FunctionTemplate(names = {"repeat", "repeatstr"}, scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                    outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_FIXED_WIDTH_DEFUALT)
   public static class RepeatString implements DrillSimpleFunc {
 
     @Param  VarCharHolder in;
@@ -1666,7 +1698,8 @@ public void eval() {
   /**
   * Convert string to ASCII from another encoding input.
   */
-  @FunctionTemplate(name = "toascii", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  @FunctionTemplate(name = "toascii", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                    outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_CLONE_DEFAULT)
   public static class AsciiEndode implements DrillSimpleFunc {
     @Param  VarCharHolder in;
     @Param  VarCharHolder enc;
@@ -1700,7 +1733,8 @@ public void eval() {
   @FunctionTemplate(name = "reverse",
       scope = FunctionScope.SIMPLE,
       returnType = ReturnType.SAME_IN_OUT_LENGTH,
-      nulls = NullHandling.NULL_IF_NULL)
+      nulls = NullHandling.NULL_IF_NULL,
+      outputWidthCalculatorType = OutputWidthCalculatorType.CLONE)
   public static class ReverseString implements DrillSimpleFunc {
     @Param  VarCharHolder in;
     @Output VarCharHolder out;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BigIntBEConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BigIntBEConvertTo.java
index eacf535764..b8bd9688ff 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BigIntBEConvertTo.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BigIntBEConvertTo.java
@@ -29,8 +29,10 @@
 import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.holders.BigIntHolder;
 import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.physical.impl.project.OutputSizeEstimateConstants;
 
-@FunctionTemplate(name = "convert_toBIGINT_BE", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+@FunctionTemplate(name = "convert_toBIGINT_BE", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                  outputSizeEstimate = OutputSizeEstimateConstants.CONVERT_TO_BIGINT_LENGTH)
 public class BigIntBEConvertTo implements DrillSimpleFunc {
 
   @Param BigIntHolder in;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BigIntConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BigIntConvertTo.java
index 81834798b7..35e659bd83 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BigIntConvertTo.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BigIntConvertTo.java
@@ -29,8 +29,10 @@
 import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.holders.BigIntHolder;
 import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.physical.impl.project.OutputSizeEstimateConstants;
 
-@FunctionTemplate(name = "convert_toBIGINT", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+@FunctionTemplate(name = "convert_toBIGINT", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                  outputSizeEstimate = OutputSizeEstimateConstants.CONVERT_TO_BIGINT_LENGTH)
 public class BigIntConvertTo implements DrillSimpleFunc {
 
   @Param BigIntHolder in;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BigIntVLongConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BigIntVLongConvertTo.java
index 26c2033fdb..b6941973a6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BigIntVLongConvertTo.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BigIntVLongConvertTo.java
@@ -29,8 +29,10 @@
 import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.holders.BigIntHolder;
 import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.physical.impl.project.OutputSizeEstimateConstants;
 
-@FunctionTemplate(name = "convert_toBIGINT_HADOOPV", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+@FunctionTemplate(name = "convert_toBIGINT_HADOOPV", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                  outputSizeEstimate = OutputSizeEstimateConstants.CONVERT_TO_HADOOPV_LENGTH)
 public class BigIntVLongConvertTo implements DrillSimpleFunc {
 
   @Param BigIntHolder in;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BooleanByteConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BooleanByteConvertTo.java
index c7a96e8a64..046258c2e7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BooleanByteConvertTo.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BooleanByteConvertTo.java
@@ -29,8 +29,10 @@
 import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.holders.BitHolder;
 import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.physical.impl.project.OutputSizeEstimateConstants;
 
-@FunctionTemplate(name = "convert_toBOOLEAN_BYTE", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+@FunctionTemplate(name = "convert_toBOOLEAN_BYTE", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                  outputSizeEstimate = OutputSizeEstimateConstants.CONVERT_TO_BOOLEAN_BYTE_LENGTH)
 public class BooleanByteConvertTo implements DrillSimpleFunc {
 
   @Param BitHolder in;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DateEpochBEConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DateEpochBEConvertTo.java
index 69aab45da7..e6c88e9649 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DateEpochBEConvertTo.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DateEpochBEConvertTo.java
@@ -29,8 +29,10 @@
 import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.holders.DateHolder;
 import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.physical.impl.project.OutputSizeEstimateConstants;
 
-@FunctionTemplate(name = "convert_toDATE_EPOCH_BE", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+@FunctionTemplate(name = "convert_toDATE_EPOCH_BE", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                  outputSizeEstimate = OutputSizeEstimateConstants.CONVERT_TO_DATE_EPOCH_LENGTH)
 public class DateEpochBEConvertTo implements DrillSimpleFunc {
 
   @Param DateHolder in;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DateEpochConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DateEpochConvertTo.java
index 990ebea03d..74ec326231 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DateEpochConvertTo.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DateEpochConvertTo.java
@@ -29,8 +29,10 @@
 import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.holders.DateHolder;
 import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.physical.impl.project.OutputSizeEstimateConstants;
 
-@FunctionTemplate(name = "convert_toDATE_EPOCH", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+@FunctionTemplate(name = "convert_toDATE_EPOCH", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                  outputSizeEstimate = OutputSizeEstimateConstants.CONVERT_TO_DATE_EPOCH_LENGTH)
 public class DateEpochConvertTo implements DrillSimpleFunc {
 
   @Param DateHolder in;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DoubleBEConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DoubleBEConvertTo.java
index e0b5cb1c91..a2718e771f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DoubleBEConvertTo.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DoubleBEConvertTo.java
@@ -26,10 +26,12 @@
 import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.holders.Float8Holder;
 import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.physical.impl.project.OutputSizeEstimateConstants;
 
 import javax.inject.Inject;
 
-@FunctionTemplate(name = "convert_toDOUBLE_BE", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+@FunctionTemplate(name = "convert_toDOUBLE_BE", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                  outputSizeEstimate = OutputSizeEstimateConstants.CONVERT_TO_DOUBLE_LENGTH)
 public class DoubleBEConvertTo implements DrillSimpleFunc {
 
   @Param Float8Holder in;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DoubleConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DoubleConvertTo.java
index cd183d8d14..225d3b9a0a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DoubleConvertTo.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DoubleConvertTo.java
@@ -29,8 +29,10 @@
 import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.holders.Float8Holder;
 import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.physical.impl.project.OutputSizeEstimateConstants;
 
-@FunctionTemplate(name = "convert_toDOUBLE", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+@FunctionTemplate(name = "convert_toDOUBLE", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                  outputSizeEstimate = OutputSizeEstimateConstants.CONVERT_TO_DOUBLE_LENGTH)
 public class DoubleConvertTo implements DrillSimpleFunc {
 
   @Param Float8Holder in;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DummyConvertFrom.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DummyConvertFrom.java
index aac66155c4..50e4cf09e9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DummyConvertFrom.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DummyConvertFrom.java
@@ -28,7 +28,8 @@
  * This and {@link DummyConvertTo} class merely act as a placeholder so that Optiq
  * allows 'convert_to()' and 'convert_from()' functions in SQL.
  */
-@FunctionTemplate(name = "convert_from", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+@FunctionTemplate(name = "convert_from", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                  outputWidthCalculatorType = FunctionTemplate.OutputWidthCalculatorType.DEFAULT)
 public class DummyConvertFrom implements DrillSimpleFunc {
 
   @Output VarBinaryHolder out;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DummyConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DummyConvertTo.java
index ba350c537b..a17dbe84ea 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DummyConvertTo.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DummyConvertTo.java
@@ -28,7 +28,8 @@
  * This and {@link DummyConvertFrom} class merely act as a placeholder so that Optiq
  * allows 'convert_to()' and 'convert_from()' functions in SQL.
  */
-@FunctionTemplate(name = "convert_to", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+@FunctionTemplate(name = "convert_to", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                  outputWidthCalculatorType = FunctionTemplate.OutputWidthCalculatorType.DEFAULT)
 public class DummyConvertTo implements DrillSimpleFunc {
 
   @Output VarBinaryHolder out;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/FloatBEConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/FloatBEConvertTo.java
index 0589d8d14e..e03b17318d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/FloatBEConvertTo.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/FloatBEConvertTo.java
@@ -26,10 +26,12 @@
 import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.holders.Float4Holder;
 import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.physical.impl.project.OutputSizeEstimateConstants;
 
 import javax.inject.Inject;
 
-@FunctionTemplate(name = "convert_toFLOAT_BE", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+@FunctionTemplate(name = "convert_toFLOAT_BE", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                  outputSizeEstimate = OutputSizeEstimateConstants.CONVERT_TO_FLOAT_LENGTH)
 public class FloatBEConvertTo implements DrillSimpleFunc {
 
   @Param Float4Holder in;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/FloatConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/FloatConvertTo.java
index b624471631..25f127233a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/FloatConvertTo.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/FloatConvertTo.java
@@ -29,8 +29,10 @@
 import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.holders.Float4Holder;
 import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.physical.impl.project.OutputSizeEstimateConstants;
 
-@FunctionTemplate(name = "convert_toFLOAT", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+@FunctionTemplate(name = "convert_toFLOAT", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                  outputSizeEstimate = OutputSizeEstimateConstants.CONVERT_TO_FLOAT_LENGTH)
 public class FloatConvertTo implements DrillSimpleFunc {
 
   @Param Float4Holder in;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/IntBEConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/IntBEConvertTo.java
index 2f1751a8fe..b33c1592ff 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/IntBEConvertTo.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/IntBEConvertTo.java
@@ -29,8 +29,10 @@
 import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.holders.IntHolder;
 import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.physical.impl.project.OutputSizeEstimateConstants;
 
-@FunctionTemplate(name = "convert_toINT_BE", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+@FunctionTemplate(name = "convert_toINT_BE", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                  outputSizeEstimate = OutputSizeEstimateConstants.CONVERT_TO_INT_LENGTH)
 public class IntBEConvertTo implements DrillSimpleFunc {
 
   @Param IntHolder in;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/IntConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/IntConvertTo.java
index a32ab4dc9e..a9f7d8fb25 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/IntConvertTo.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/IntConvertTo.java
@@ -29,8 +29,10 @@
 import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.holders.IntHolder;
 import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.physical.impl.project.OutputSizeEstimateConstants;
 
-@FunctionTemplate(name = "convert_toINT", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+@FunctionTemplate(name = "convert_toINT", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                  outputSizeEstimate = OutputSizeEstimateConstants.CONVERT_TO_INT_LENGTH)
 public class IntConvertTo implements DrillSimpleFunc {
 
   @Param IntHolder in;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/IntVIntConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/IntVIntConvertTo.java
index bc0347d5db..ed0a269529 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/IntVIntConvertTo.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/IntVIntConvertTo.java
@@ -29,8 +29,10 @@
 import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.holders.IntHolder;
 import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.physical.impl.project.OutputSizeEstimateConstants;
 
-@FunctionTemplate(name = "convert_toINT_HADOOPV", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+@FunctionTemplate(name = "convert_toINT_HADOOPV", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                  outputSizeEstimate = OutputSizeEstimateConstants.CONVERT_TO_HADOOPV_LENGTH)
 public class IntVIntConvertTo implements DrillSimpleFunc {
 
   @Param IntHolder in;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/JsonConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/JsonConvertTo.java
index 68fdb28534..e384892365 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/JsonConvertTo.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/JsonConvertTo.java
@@ -44,7 +44,8 @@
 
   private JsonConvertTo(){}
 
-  @FunctionTemplate(names = { "convert_toJSON", "convert_toSIMPLEJSON" } , scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  @FunctionTemplate(names = { "convert_toJSON", "convert_toSIMPLEJSON" } , scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                    outputWidthCalculatorType = FunctionTemplate.OutputWidthCalculatorType.CUSTOM_FIXED_WIDTH_DEFUALT)
   public static class ConvertToJson implements DrillSimpleFunc{
 
     @Param FieldReader input;
@@ -74,7 +75,8 @@ public void eval(){
     }
   }
 
-  @FunctionTemplate(name = "convert_toEXTENDEDJSON", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  @FunctionTemplate(name = "convert_toEXTENDEDJSON", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                    outputWidthCalculatorType = FunctionTemplate.OutputWidthCalculatorType.CUSTOM_FIXED_WIDTH_DEFUALT)
   public static class ConvertToExtendedJson implements DrillSimpleFunc{
 
     @Param FieldReader input;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/SmallIntBEConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/SmallIntBEConvertTo.java
index 420fe3452c..0b143137d4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/SmallIntBEConvertTo.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/SmallIntBEConvertTo.java
@@ -29,8 +29,10 @@
 import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.holders.SmallIntHolder;
 import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.physical.impl.project.OutputSizeEstimateConstants;
 
-@FunctionTemplate(name = "convert_toSMALLINT_BE", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+@FunctionTemplate(name = "convert_toSMALLINT_BE", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+outputSizeEstimate = OutputSizeEstimateConstants.CONVERT_TO_SMALLINT_LENGTH)
 public class SmallIntBEConvertTo implements DrillSimpleFunc {
 
   @Param SmallIntHolder in;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/SmallIntConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/SmallIntConvertTo.java
index c228d9655c..bee0c649a5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/SmallIntConvertTo.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/SmallIntConvertTo.java
@@ -29,8 +29,10 @@
 import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.holders.SmallIntHolder;
 import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.physical.impl.project.OutputSizeEstimateConstants;
 
-@FunctionTemplate(name = "convert_toSMALLINT", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+@FunctionTemplate(name = "convert_toSMALLINT", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                  outputSizeEstimate = OutputSizeEstimateConstants.CONVERT_TO_SMALLINT_LENGTH)
 public class SmallIntConvertTo implements DrillSimpleFunc {
 
   @Param SmallIntHolder in;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/TimeEpochBEConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/TimeEpochBEConvertTo.java
index 8effc8992c..a7a08f2e2d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/TimeEpochBEConvertTo.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/TimeEpochBEConvertTo.java
@@ -29,8 +29,10 @@
 import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.holders.TimeHolder;
 import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.physical.impl.project.OutputSizeEstimateConstants;
 
-@FunctionTemplate(name = "convert_toTIME_EPOCH_BE", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+@FunctionTemplate(name = "convert_toTIME_EPOCH_BE", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+        outputSizeEstimate = OutputSizeEstimateConstants.CONVERT_TO_TIME_EPOCH_LENGTH)
 public class TimeEpochBEConvertTo implements DrillSimpleFunc {
 
   @Param TimeHolder in;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/TimeEpochConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/TimeEpochConvertTo.java
index 39fe3b687e..c17efbf7a5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/TimeEpochConvertTo.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/TimeEpochConvertTo.java
@@ -29,8 +29,10 @@
 import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.holders.TimeHolder;
 import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.physical.impl.project.OutputSizeEstimateConstants;
 
-@FunctionTemplate(name = "convert_toTIME_EPOCH", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+@FunctionTemplate(name = "convert_toTIME_EPOCH", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                  outputSizeEstimate = OutputSizeEstimateConstants.CONVERT_TO_TIME_EPOCH_LENGTH)
 public class TimeEpochConvertTo implements DrillSimpleFunc {
 
   @Param TimeHolder in;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/TimeStampEpochBEConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/TimeStampEpochBEConvertTo.java
index a34c6f8dde..98a4296201 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/TimeStampEpochBEConvertTo.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/TimeStampEpochBEConvertTo.java
@@ -29,8 +29,10 @@
 import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.holders.TimeStampHolder;
 import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.physical.impl.project.OutputSizeEstimateConstants;
 
-@FunctionTemplate(name = "convert_toTIMESTAMP_EPOCH_BE", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+@FunctionTemplate(name = "convert_toTIMESTAMP_EPOCH_BE", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+outputSizeEstimate = OutputSizeEstimateConstants.CONVERT_TO_TIMESTAMP_EPOCH_LENGTH)
 public class TimeStampEpochBEConvertTo implements DrillSimpleFunc {
 
   @Param TimeStampHolder in;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/TinyIntConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/TinyIntConvertTo.java
index fa7ccb09e0..3ca23dbf3d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/TinyIntConvertTo.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/TinyIntConvertTo.java
@@ -29,9 +29,11 @@
 import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.holders.TinyIntHolder;
 import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.physical.impl.project.OutputSizeEstimateConstants;
 
 @FunctionTemplate(names = {"convert_toTINYINT", "convert_toTINYINT_BE"},
-  scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+        outputSizeEstimate = OutputSizeEstimateConstants.CONVERT_TO_TINYINT_LENGTH)
 public class TinyIntConvertTo implements DrillSimpleFunc {
 
   @Param TinyIntHolder in;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/UInt4BEConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/UInt4BEConvertTo.java
index c18d2817a0..bda7175a89 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/UInt4BEConvertTo.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/UInt4BEConvertTo.java
@@ -29,8 +29,10 @@
 import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.holders.UInt4Holder;
 import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.physical.impl.project.OutputSizeEstimateConstants;
 
-@FunctionTemplate(name = "convert_toUINT4_BE", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+@FunctionTemplate(name = "convert_toUINT4_BE", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                  outputSizeEstimate = OutputSizeEstimateConstants.CONVERT_TO_UINT4_LENGTH)
 public class UInt4BEConvertTo implements DrillSimpleFunc {
 
   @Param UInt4Holder in;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/UInt4ConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/UInt4ConvertTo.java
index ea574c21c3..b3777a8451 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/UInt4ConvertTo.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/UInt4ConvertTo.java
@@ -29,8 +29,10 @@
 import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.holders.UInt4Holder;
 import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.physical.impl.project.OutputSizeEstimateConstants;
 
-@FunctionTemplate(name = "convert_toUINT4", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+@FunctionTemplate(name = "convert_toUINT4", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                   outputSizeEstimate = OutputSizeEstimateConstants.CONVERT_TO_UINT4_LENGTH)
 public class UInt4ConvertTo implements DrillSimpleFunc {
 
   @Param UInt4Holder in;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/UInt8ConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/UInt8ConvertTo.java
index 446fe72e50..46f38847ee 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/UInt8ConvertTo.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/UInt8ConvertTo.java
@@ -29,8 +29,10 @@
 import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.holders.UInt8Holder;
 import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.physical.impl.project.OutputSizeEstimateConstants;
 
-@FunctionTemplate(name = "convert_toUINT8", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+@FunctionTemplate(name = "convert_toUINT8", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                  outputSizeEstimate = OutputSizeEstimateConstants.CONVERT_TO_UINT8_LENGTH)
 public class UInt8ConvertTo implements DrillSimpleFunc {
 
   @Param UInt8Holder in;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/UTF16ConvertFrom.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/UTF16ConvertFrom.java
index baad4f82e1..298213e3b7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/UTF16ConvertFrom.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/UTF16ConvertFrom.java
@@ -26,7 +26,8 @@
 import org.apache.drill.exec.expr.holders.Var16CharHolder;
 import org.apache.drill.exec.expr.holders.VarBinaryHolder;
 
-@FunctionTemplate(name = "convert_fromUTF16", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+@FunctionTemplate(name = "convert_fromUTF16", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                  outputWidthCalculatorType = FunctionTemplate.OutputWidthCalculatorType.CLONE)
 public class UTF16ConvertFrom implements DrillSimpleFunc {
 
   @Param VarBinaryHolder in;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/UTF16ConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/UTF16ConvertTo.java
index 71dddb3556..0cccb94e2f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/UTF16ConvertTo.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/UTF16ConvertTo.java
@@ -19,6 +19,7 @@
 
 import org.apache.drill.exec.expr.DrillSimpleFunc;
 import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.OutputWidthCalculatorType;
 import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
 import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
 import org.apache.drill.exec.expr.annotations.Output;
@@ -26,7 +27,8 @@
 import org.apache.drill.exec.expr.holders.Var16CharHolder;
 import org.apache.drill.exec.expr.holders.VarBinaryHolder;
 
-@FunctionTemplate(name = "convert_toUTF16", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+@FunctionTemplate(name = "convert_toUTF16", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                  outputWidthCalculatorType = OutputWidthCalculatorType.CLONE)
 public class UTF16ConvertTo implements DrillSimpleFunc {
 
   @Param Var16CharHolder in;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/UTF8ConvertFrom.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/UTF8ConvertFrom.java
index 08de915b24..68a7320db5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/UTF8ConvertFrom.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/UTF8ConvertFrom.java
@@ -26,7 +26,8 @@
 import org.apache.drill.exec.expr.holders.VarBinaryHolder;
 import org.apache.drill.exec.expr.holders.VarCharHolder;
 
-@FunctionTemplate(name = "convert_fromUTF8", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+@FunctionTemplate(name = "convert_fromUTF8", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                  outputWidthCalculatorType = FunctionTemplate.OutputWidthCalculatorType.CLONE)
 public class UTF8ConvertFrom implements DrillSimpleFunc {
 
   @Param VarBinaryHolder in;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/UTF8ConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/UTF8ConvertTo.java
index 1eef114ef0..fb7ad63ce7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/UTF8ConvertTo.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/UTF8ConvertTo.java
@@ -26,7 +26,8 @@
 import org.apache.drill.exec.expr.holders.VarBinaryHolder;
 import org.apache.drill.exec.expr.holders.VarCharHolder;
 
-@FunctionTemplate(name = "convert_toUTF8", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+@FunctionTemplate(name = "convert_toUTF8", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
+                  outputWidthCalculatorType = FunctionTemplate.OutputWidthCalculatorType.CLONE)
 public class UTF8ConvertTo implements DrillSimpleFunc {
 
   @Param VarCharHolder in;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/output/OutputWidthCalculator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/output/OutputWidthCalculator.java
new file mode 100644
index 0000000000..6eadbf5642
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/output/OutputWidthCalculator.java
@@ -0,0 +1,33 @@
+/*
+ * 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.fn.output;
+
+import org.apache.drill.exec.physical.impl.project.OutputWidthExpression.FixedLenExpr;
+
+import java.util.List;
+
+/**
+ * Return type calculation interface for functions that have return type set as with enum
+ * {@link org.apache.drill.exec.expr.annotations.FunctionTemplate.ReturnType}.
+ */
+public interface OutputWidthCalculator {
+
+  int getOutputWidth(List<FixedLenExpr> args);
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/output/OutputWidthCalculators.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/output/OutputWidthCalculators.java
new file mode 100644
index 0000000000..7c0bb380aa
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/output/OutputWidthCalculators.java
@@ -0,0 +1,113 @@
+/*
+ * 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.fn.output;
+
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.physical.impl.project.OutputWidthExpression.FixedLenExpr;
+
+import java.util.List;
+
+/**
+ * Return type calculation implementation for functions with return type set as
+ * {@link org.apache.drill.exec.expr.annotations.FunctionTemplate.ReturnType#CONCAT}.
+ */
+
+public class OutputWidthCalculators {
+
+    private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(OutputWidthCalculators.class);
+
+    private static int adjustOutputWidth(int outputSize, String prefix) {
+        if (outputSize > Types.MAX_VARCHAR_LENGTH || outputSize < 0 /*overflow*/) {
+            logger.warn(prefix + " Output size for expressions is too large, setting to MAX_VARCHAR_LENGTH");
+            outputSize = Types.MAX_VARCHAR_LENGTH;
+        }
+        return outputSize;
+    }
+
+    public static class ConcatOutputWidthCalculator implements OutputWidthCalculator {
+
+        public static final ConcatOutputWidthCalculator INSTANCE = new ConcatOutputWidthCalculator();
+
+        /**
+         * Defines a function's output size estimate as sum of input sizes
+         * If calculated size is greater than {@link Types#MAX_VARCHAR_LENGTH},
+         * it is replaced with {@link Types#MAX_VARCHAR_LENGTH}.
+         *
+         * @param args
+         * @return return type
+         */
+        @Override
+        public int getOutputWidth(List<FixedLenExpr> args) {
+            int outputSize = 0;
+            if (args == null || args.size() == 0) {
+                throw new IllegalArgumentException();
+            }
+            for (FixedLenExpr expr : args) {
+                outputSize += expr.getWidth();
+            }
+            outputSize = adjustOutputWidth(outputSize, "ConcatOutputWidthCalculator:");
+            return outputSize;
+        }
+    }
+
+    public static class CloneOutputWidthCalculator implements OutputWidthCalculator {
+
+        public static final CloneOutputWidthCalculator INSTANCE = new CloneOutputWidthCalculator();
+
+        /**
+         * Defines a function's output size estimate as the same length as the first
+         * argument. In other words, treats the function as a CLONE function
+         * If calculated size is greater than {@link Types#MAX_VARCHAR_LENGTH},
+         * it is replaced with {@link Types#MAX_VARCHAR_LENGTH}.
+         *
+         * @param args logical expressions
+         * @return return type
+         */
+        @Override
+        public int getOutputWidth(List<FixedLenExpr> args) {
+            int outputSize = 0;
+            if (args == null || args.size() < 1) {
+                throw new IllegalArgumentException();
+            }
+            outputSize = args.get(0).getWidth();
+            outputSize = adjustOutputWidth(outputSize, "CloneOutputWidthCalculator:");
+            return outputSize;
+        }
+    }
+
+    public static class DefaultOutputWidthCalculator implements OutputWidthCalculator {
+
+        public static final DefaultOutputWidthCalculator INSTANCE = new DefaultOutputWidthCalculator();
+
+        /**
+         * Defines a function's output size estimate as some fixed value specified via an option
+         * If calculated size is greater than {@link Types#MAX_VARCHAR_LENGTH},
+         * it is replaced with {@link Types#MAX_VARCHAR_LENGTH}.
+         *
+         * @param args logical expressions
+         * @return return type
+         */
+        @Override
+        public int getOutputWidth(List<FixedLenExpr> args) {
+            //TODO: Read value from options?
+            int outputSize = adjustOutputWidth(50, "DefaultOutputWidthCalculator:");
+            return outputSize;
+        }
+    }
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetBooleanPredicate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetBooleanPredicate.java
new file mode 100644
index 0000000000..fa5c4672a8
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetBooleanPredicate.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.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 abstract class ParquetBooleanPredicate<C extends Comparable<C>> extends BooleanOperator
+    implements ParquetFilterPredicate<C> {
+
+  private 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);
+  }
+
+  @SuppressWarnings("unchecked")
+  private static <C extends Comparable<C>> LogicalExpression createAndPredicate(
+      String name,
+      List<LogicalExpression> args,
+      ExpressionPosition pos
+  ) {
+    return new ParquetBooleanPredicate<C>(name, args, pos) {
+      @Override
+      public boolean canDrop(RangeExprEvaluator<C> 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;
+      }
+    };
+  }
+
+  @SuppressWarnings("unchecked")
+  private static <C extends Comparable<C>> LogicalExpression createOrPredicate(
+      String name,
+      List<LogicalExpression> args,
+      ExpressionPosition pos
+  ) {
+    return new ParquetBooleanPredicate<C>(name, args, pos) {
+      @Override
+      public boolean canDrop(RangeExprEvaluator<C> evaluator) {
+        for (LogicalExpression child : this) {
+          // "or" : as long as one branch is NOT ok to drop, we can NOT drop it.
+          if (!(child instanceof ParquetFilterPredicate) || !((ParquetFilterPredicate)child).canDrop(evaluator)) {
+            return false;
+          }
+        }
+        return true;
+      }
+    };
+  }
+
+  public static <C extends Comparable<C>> LogicalExpression createBooleanPredicate(
+      String function,
+      String name,
+      List<LogicalExpression> args,
+      ExpressionPosition pos
+  ) {
+    switch (function) {
+      case "booleanOr":
+        return ParquetBooleanPredicate.<C>createOrPredicate(name, args, pos);
+      case "booleanAnd":
+        return ParquetBooleanPredicate.<C>createAndPredicate(name, args, pos);
+      default:
+        logger.warn("Unknown Boolean '{}' predicate.", function);
+        return null;
+    }
+  }
+}
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
deleted file mode 100644
index e5de34fc9d..0000000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetBooleanPredicates.java
+++ /dev/null
@@ -1,76 +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.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;
-    }
-  }
-}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetComparisonPredicate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetComparisonPredicate.java
new file mode 100644
index 0000000000..ebceefb435
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetComparisonPredicate.java
@@ -0,0 +1,218 @@
+/*
+ * 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.drill.exec.expr.fn.FunctionGenerationHelper;
+import org.apache.parquet.column.statistics.Statistics;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.function.BiPredicate;
+
+import static org.apache.drill.exec.expr.stat.ParquetPredicatesHelper.isNullOrEmpty;
+import static org.apache.drill.exec.expr.stat.ParquetPredicatesHelper.isAllNulls;
+
+/**
+ * Comparison predicates for parquet filter pushdown.
+ */
+public class ParquetComparisonPredicate<C extends Comparable<C>> extends LogicalExpressionBase
+    implements ParquetFilterPredicate<C> {
+  private final LogicalExpression left;
+  private final LogicalExpression right;
+  private final BiPredicate<Statistics<C>, Statistics<C>> predicate;
+
+  private ParquetComparisonPredicate(
+      LogicalExpression left,
+      LogicalExpression right,
+      BiPredicate<Statistics<C>, Statistics<C>> predicate
+  ) {
+    super(left.getPosition());
+    this.left = left;
+    this.right = right;
+    this.predicate = predicate;
+  }
+
+  @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);
+  }
+
+  /**
+   * 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<C> evaluator) {
+    Statistics<C> leftStat = left.accept(evaluator, null);
+    if (isNullOrEmpty(leftStat)) {
+      return false;
+    }
+
+    Statistics<C> rightStat = right.accept(evaluator, null);
+    if (isNullOrEmpty(rightStat)) {
+      return false;
+    }
+
+    // if either side is ALL null, = is evaluated to UNKNOWN -> canDrop
+    if (isAllNulls(leftStat, evaluator.getRowCount()) || isAllNulls(rightStat, evaluator.getRowCount())) {
+      return true;
+    }
+
+    return (leftStat.hasNonNullValue() && rightStat.hasNonNullValue()) && predicate.test(leftStat, rightStat);
+  }
+
+  /**
+   * EQ (=) predicate
+   */
+  private static <C extends Comparable<C>> LogicalExpression createEqualPredicate(
+      LogicalExpression left,
+      LogicalExpression right
+  ) {
+    return new ParquetComparisonPredicate<C>(left, right, (leftStat, rightStat) -> {
+      // can drop when left's max < right's min, or right's max < left's min
+      final C leftMin = leftStat.genericGetMin();
+      final C rightMin = rightStat.genericGetMin();
+      return (leftStat.compareMaxToValue(rightMin) < 0) || (rightStat.compareMaxToValue(leftMin) < 0);
+    }) {
+      @Override
+      public String toString() {
+        return left + " = " + right;
+      }
+    };
+  }
+
+  /**
+   * GT (>) predicate.
+   */
+  private static <C extends Comparable<C>> LogicalExpression createGTPredicate(
+      LogicalExpression left,
+      LogicalExpression right
+  ) {
+    return new ParquetComparisonPredicate<C>(left, right, (leftStat, rightStat) -> {
+      // can drop when left's max <= right's min.
+      final C rightMin = rightStat.genericGetMin();
+      return leftStat.compareMaxToValue(rightMin) <= 0;
+    });
+  }
+
+  /**
+   * GE (>=) predicate.
+   */
+  private static <C extends Comparable<C>> LogicalExpression createGEPredicate(
+      LogicalExpression left,
+      LogicalExpression right
+  ) {
+    return new ParquetComparisonPredicate<C>(left, right, (leftStat, rightStat) -> {
+      // can drop when left's max < right's min.
+      final C rightMin = rightStat.genericGetMin();
+      return leftStat.compareMaxToValue(rightMin) < 0;
+    });
+  }
+
+  /**
+   * LT (<) predicate.
+   */
+  private static <C extends Comparable<C>> LogicalExpression createLTPredicate(
+      LogicalExpression left,
+      LogicalExpression right
+  ) {
+    return new ParquetComparisonPredicate<C>(left, right, (leftStat, rightStat) -> {
+      // can drop when right's max <= left's min.
+      final C leftMin = leftStat.genericGetMin();
+      return rightStat.compareMaxToValue(leftMin) <= 0;
+    });
+  }
+
+  /**
+   * LE (<=) predicate.
+   */
+  private static <C extends Comparable<C>> LogicalExpression createLEPredicate(
+      LogicalExpression left, LogicalExpression right
+  ) {
+    return new ParquetComparisonPredicate<C>(left, right, (leftStat, rightStat) -> {
+      // can drop when right's max < left's min.
+      final C leftMin = leftStat.genericGetMin();
+      return rightStat.compareMaxToValue(leftMin) < 0;
+    });
+  }
+
+  /**
+   * NE (!=) predicate.
+   */
+  private static <C extends Comparable<C>> LogicalExpression createNEPredicate(
+      LogicalExpression left,
+      LogicalExpression right
+  ) {
+    return new ParquetComparisonPredicate<C>(left, right, (leftStat, rightStat) -> {
+      // can drop when there is only one unique value.
+      final C leftMax = leftStat.genericGetMax();
+      final C rightMax = rightStat.genericGetMax();
+      return leftStat.compareMinToValue(leftMax) == 0 && rightStat.compareMinToValue(rightMax) == 0 &&
+          leftStat.compareMaxToValue(rightMax) == 0;
+    });
+  }
+
+  public static <C extends Comparable<C>> LogicalExpression createComparisonPredicate(
+      String function,
+      LogicalExpression left,
+      LogicalExpression right
+  ) {
+    switch (function) {
+      case FunctionGenerationHelper.EQ:
+        return ParquetComparisonPredicate.<C>createEqualPredicate(left, right);
+      case FunctionGenerationHelper.GT:
+        return ParquetComparisonPredicate.<C>createGTPredicate(left, right);
+      case FunctionGenerationHelper.GE:
+        return ParquetComparisonPredicate.<C>createGEPredicate(left, right);
+      case FunctionGenerationHelper.LT:
+        return ParquetComparisonPredicate.<C>createLTPredicate(left, right);
+      case FunctionGenerationHelper.LE:
+        return ParquetComparisonPredicate.<C>createLEPredicate(left, right);
+      case FunctionGenerationHelper.NE:
+        return ParquetComparisonPredicate.<C>createNEPredicate(left, right);
+      default:
+        return null;
+    }
+  }
+}
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
deleted file mode 100644
index 5ba597c2a1..0000000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetComparisonPredicates.java
+++ /dev/null
@@ -1,293 +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.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;
-      }
-    }
-  }
-}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetFilterPredicate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetFilterPredicate.java
index 898dc71db6..1b7e9e5934 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetFilterPredicate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetFilterPredicate.java
@@ -17,6 +17,6 @@
  */
 package org.apache.drill.exec.expr.stat;
 
-public interface ParquetFilterPredicate {
-  boolean canDrop(RangeExprEvaluator evaluator);
+public interface ParquetFilterPredicate<T extends Comparable<T>> {
+  boolean canDrop(RangeExprEvaluator<T> evaluator);
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetIsPredicate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetIsPredicate.java
new file mode 100644
index 0000000000..42e6e0b6a4
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetIsPredicate.java
@@ -0,0 +1,171 @@
+/*
+ * 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.SchemaPath;
+import org.apache.drill.common.expression.TypedFieldExpr;
+import org.apache.drill.common.expression.visitors.ExprVisitor;
+import org.apache.drill.exec.expr.fn.FunctionGenerationHelper;
+import org.apache.parquet.column.statistics.BooleanStatistics;
+import org.apache.parquet.column.statistics.Statistics;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.function.BiPredicate;
+
+import static org.apache.drill.exec.expr.stat.ParquetPredicatesHelper.hasNoNulls;
+import static org.apache.drill.exec.expr.stat.ParquetPredicatesHelper.isAllNulls;
+import static org.apache.drill.exec.expr.stat.ParquetPredicatesHelper.isNullOrEmpty;
+
+/**
+ * IS predicates for parquet filter pushdown.
+ */
+public class ParquetIsPredicate<C extends Comparable<C>> extends LogicalExpressionBase
+    implements ParquetFilterPredicate<C> {
+
+  private final LogicalExpression expr;
+  private final BiPredicate<Statistics<C>, RangeExprEvaluator<C>> predicate;
+
+  private ParquetIsPredicate(LogicalExpression expr, BiPredicate<Statistics<C>, RangeExprEvaluator<C>> predicate) {
+    super(expr.getPosition());
+    this.expr = expr;
+    this.predicate = predicate;
+  }
+
+  @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);
+  }
+
+  @Override
+  public boolean canDrop(RangeExprEvaluator<C> evaluator) {
+    Statistics<C> exprStat = expr.accept(evaluator, null);
+    if (isNullOrEmpty(exprStat)) {
+      return false;
+    }
+
+    return predicate.test(exprStat, evaluator);
+  }
+
+  /**
+   * IS NULL predicate.
+   */
+  private static <C extends Comparable<C>> LogicalExpression createIsNullPredicate(LogicalExpression expr) {
+    return new ParquetIsPredicate<C>(expr,
+        //if there are no nulls  -> canDrop
+        (exprStat, evaluator) -> hasNoNulls(exprStat)) {
+      private final boolean isArray = isArray(expr);
+
+      private boolean isArray(LogicalExpression expression) {
+        if (expression instanceof TypedFieldExpr) {
+          TypedFieldExpr typedFieldExpr = (TypedFieldExpr) expression;
+          SchemaPath schemaPath = typedFieldExpr.getPath();
+          return schemaPath.isArray();
+        }
+        return false;
+      }
+
+      @Override
+      public boolean canDrop(RangeExprEvaluator<C> evaluator) {
+        // for arrays we are not able to define exact number of nulls
+        // [1,2,3] vs [1,2] -> in second case 3 is absent and thus it's null but statistics shows no nulls
+        return !isArray && super.canDrop(evaluator);
+      }
+    };
+  }
+
+  /**
+   * IS NOT NULL predicate.
+   */
+  private static <C extends Comparable<C>> LogicalExpression createIsNotNullPredicate(LogicalExpression expr) {
+    return new ParquetIsPredicate<C>(expr,
+        //if there are all nulls  -> canDrop
+        (exprStat, evaluator) -> isAllNulls(exprStat, evaluator.getRowCount())
+    );
+  }
+
+  /**
+   * IS TRUE predicate.
+   */
+  private static LogicalExpression createIsTruePredicate(LogicalExpression expr) {
+    return new ParquetIsPredicate<Boolean>(expr, (exprStat, evaluator) ->
+        //if max value is not true or if there are all nulls  -> canDrop
+        isAllNulls(exprStat, evaluator.getRowCount()) || exprStat.hasNonNullValue() && !((BooleanStatistics) exprStat).getMax()
+    );
+  }
+
+  /**
+   * IS FALSE predicate.
+   */
+  private static LogicalExpression createIsFalsePredicate(LogicalExpression expr) {
+    return new ParquetIsPredicate<Boolean>(expr, (exprStat, evaluator) ->
+        //if min value is not false or if there are all nulls  -> canDrop
+        isAllNulls(exprStat, evaluator.getRowCount()) || exprStat.hasNonNullValue() && ((BooleanStatistics) exprStat).getMin()
+    );
+  }
+
+  /**
+   * IS NOT TRUE predicate.
+   */
+  private static LogicalExpression createIsNotTruePredicate(LogicalExpression expr) {
+    return new ParquetIsPredicate<Boolean>(expr, (exprStat, evaluator) ->
+        //if min value is not false or if there are no nulls  -> canDrop
+        hasNoNulls(exprStat) && exprStat.hasNonNullValue() && ((BooleanStatistics) exprStat).getMin()
+    );
+  }
+
+  /**
+   * IS NOT FALSE predicate.
+   */
+  private static LogicalExpression createIsNotFalsePredicate(LogicalExpression expr) {
+    return new ParquetIsPredicate<Boolean>(expr, (exprStat, evaluator) ->
+        //if max value is not true or if there are no nulls  -> canDrop
+        hasNoNulls(exprStat) && exprStat.hasNonNullValue() && !((BooleanStatistics) exprStat).getMax()
+    );
+  }
+
+  public static <C extends Comparable<C>> LogicalExpression createIsPredicate(String function, LogicalExpression expr) {
+    switch (function) {
+      case FunctionGenerationHelper.IS_NULL:
+        return ParquetIsPredicate.<C>createIsNullPredicate(expr);
+      case FunctionGenerationHelper.IS_NOT_NULL:
+        return ParquetIsPredicate.<C>createIsNotNullPredicate(expr);
+      case FunctionGenerationHelper.IS_TRUE:
+        return createIsTruePredicate(expr);
+      case FunctionGenerationHelper.IS_NOT_TRUE:
+        return createIsNotTruePredicate(expr);
+      case FunctionGenerationHelper.IS_FALSE:
+        return createIsFalsePredicate(expr);
+      case FunctionGenerationHelper.IS_NOT_FALSE:
+        return createIsNotFalsePredicate(expr);
+      default:
+        logger.warn("Unhandled IS function. Function name: {}", function);
+        return null;
+    }
+  }
+}
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
deleted file mode 100644
index ef2b9406b2..0000000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetIsPredicates.java
+++ /dev/null
@@ -1,228 +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.expr.stat;
-
-import org.apache.drill.common.expression.LogicalExpression;
-import org.apache.drill.common.expression.LogicalExpressionBase;
-import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.common.expression.TypedFieldExpr;
-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 {
-    private final boolean isArray;
-
-    public IsNullPredicate(LogicalExpression expr) {
-      super(expr);
-      this.isArray = isArray(expr);
-    }
-
-    @Override
-    public boolean canDrop(RangeExprEvaluator evaluator) {
-
-      // for arrays we are not able to define exact number of nulls
-      // [1,2,3] vs [1,2] -> in second case 3 is absent and thus it's null but statistics shows no nulls
-      if (isArray) {
-        return false;
-      }
-
-      Statistics exprStat = expr.accept(evaluator, null);
-
-      if (!ParquetPredicatesHelper.hasStats(exprStat)) {
-        return false;
-      }
-
-      //if there are no nulls  -> canDrop
-      if (!ParquetPredicatesHelper.hasNulls(exprStat)) {
-        return true;
-      } else {
-        return false;
-      }
-    }
-
-    private boolean isArray(LogicalExpression expression) {
-      if (expression instanceof TypedFieldExpr) {
-        TypedFieldExpr typedFieldExpr = (TypedFieldExpr) expression;
-        SchemaPath schemaPath = typedFieldExpr.getPath();
-        return schemaPath.isArray();
-      }
-      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 (!ParquetPredicatesHelper.hasStats(exprStat)) {
-        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 (!ParquetPredicatesHelper.hasStats(exprStat)) {
-        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 (!ParquetPredicatesHelper.hasStats(exprStat)) {
-        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 (!ParquetPredicatesHelper.hasStats(exprStat)) {
-        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 (!ParquetPredicatesHelper.hasStats(exprStat)) {
-        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;
-      }
-    }
-  }
-}
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
index e83d393e2a..de4df1f5b9 100644
--- 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
@@ -17,20 +17,22 @@
  */
 package org.apache.drill.exec.expr.stat;
 
+import org.apache.parquet.Preconditions;
 import org.apache.parquet.column.statistics.Statistics;
 
 /**
  * Parquet predicates class helper for filter pushdown.
  */
-@SuppressWarnings("rawtypes")
-public class ParquetPredicatesHelper {
+class ParquetPredicatesHelper {
+  private ParquetPredicatesHelper() {
+  }
 
   /**
    * @param stat statistics object
-   * @return true if the input stat object has valid statistics; false otherwise
+   * @return <tt>true</tt> if the input stat object has valid statistics; false otherwise
    */
-  public static boolean hasStats(Statistics stat) {
-    return stat != null && !stat.isEmpty();
+  static boolean isNullOrEmpty(Statistics stat) {
+    return stat == null || stat.isEmpty();
   }
 
   /**
@@ -38,22 +40,21 @@ public static boolean hasStats(Statistics stat) {
    *
    * @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.
+   * @return <tt>true</tt> if all rows are null in the parquet file and <tt>false</tt> otherwise
    */
-  public static boolean isAllNulls(Statistics stat, long rowCount) {
+  static boolean isAllNulls(Statistics stat, long rowCount) {
+    Preconditions.checkArgument(rowCount >= 0, String.format("negative rowCount %d is not valid", rowCount));
     return stat.getNumNulls() == rowCount;
   }
 
   /**
-   * Checks that column chunk's statistics has at least one null
+   * Checks that column chunk's statistics does not have nulls
    *
    * @param stat parquet column statistics
-   * @return True if the parquet file has nulls
-   *          False if the parquet file hasn't nulls.
+   * @return <tt>true</tt> if the parquet file does not have nulls and <tt>false</tt> otherwise
    */
-  public static boolean hasNulls(Statistics stat) {
-    return stat.getNumNulls() > 0;
+  static boolean hasNoNulls(Statistics stat) {
+    return stat.getNumNulls() <= 0;
   }
 
 }
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 d2fa0cf01b..f127f0b5b9 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
@@ -45,6 +45,8 @@
 import org.apache.parquet.column.statistics.IntStatistics;
 import org.apache.parquet.column.statistics.LongStatistics;
 import org.apache.parquet.column.statistics.Statistics;
+import org.apache.parquet.schema.PrimitiveType;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -53,13 +55,13 @@
 import java.util.Map;
 import java.util.Set;
 
-public class RangeExprEvaluator extends AbstractExprVisitor<Statistics, Void, RuntimeException> {
+public class RangeExprEvaluator<T extends Comparable<T>> extends AbstractExprVisitor<Statistics<T>, Void, RuntimeException> {
   static final Logger logger = LoggerFactory.getLogger(RangeExprEvaluator.class);
 
-  private final Map<SchemaPath, ColumnStatistics> columnStatMap;
+  private final Map<SchemaPath, ColumnStatistics<T>> columnStatMap;
   private final long rowCount;
 
-  public RangeExprEvaluator(final Map<SchemaPath, ColumnStatistics> columnStatMap, long rowCount) {
+  public RangeExprEvaluator(final Map<SchemaPath, ColumnStatistics<T>> columnStatMap, long rowCount) {
     this.columnStatMap = columnStatMap;
     this.rowCount = rowCount;
   }
@@ -69,70 +71,71 @@ public long getRowCount() {
   }
 
   @Override
-  public Statistics visitUnknown(LogicalExpression e, Void value) throws RuntimeException {
+  public Statistics<T> visitUnknown(LogicalExpression e, Void value) throws RuntimeException {
     // do nothing for the unknown expression
     return null;
   }
 
+  @SuppressWarnings("unchecked")
   @Override
-  public Statistics visitTypedFieldExpr(TypedFieldExpr typedFieldExpr, Void value) throws RuntimeException {
-    final ColumnStatistics columnStatistics = columnStatMap.get(typedFieldExpr.getPath());
+  public Statistics<T> visitTypedFieldExpr(TypedFieldExpr typedFieldExpr, Void value) throws RuntimeException {
+    final ColumnStatistics<T> columnStatistics = columnStatMap.get(typedFieldExpr.getPath());
     if (columnStatistics != null) {
       return columnStatistics.getStatistics();
     } else if (typedFieldExpr.getMajorType().equals(Types.OPTIONAL_INT)) {
       // field does not exist.
-      IntStatistics intStatistics = new IntStatistics();
-      intStatistics.setNumNulls(rowCount); // all values are nulls
-      return intStatistics;
+      Statistics<T> statistics = Statistics.getStatsBasedOnType(PrimitiveType.PrimitiveTypeName.INT32);
+      statistics.setNumNulls(rowCount); // all values are nulls
+      return statistics;
     }
     return null;
   }
 
   @Override
-  public Statistics visitIntConstant(ValueExpressions.IntExpression expr, Void value) throws RuntimeException {
+  public Statistics<T> visitIntConstant(ValueExpressions.IntExpression expr, Void value) throws RuntimeException {
     return getStatistics(expr.getInt());
   }
 
   @Override
-  public Statistics visitBooleanConstant(ValueExpressions.BooleanExpression expr, Void value) throws RuntimeException {
+  public Statistics<T> visitBooleanConstant(ValueExpressions.BooleanExpression expr, Void value) throws RuntimeException {
     return getStatistics(expr.getBoolean());
   }
 
   @Override
-  public Statistics visitLongConstant(ValueExpressions.LongExpression expr, Void value) throws RuntimeException {
+  public Statistics<T> visitLongConstant(ValueExpressions.LongExpression expr, Void value) throws RuntimeException {
     return getStatistics(expr.getLong());
   }
 
   @Override
-  public Statistics visitFloatConstant(ValueExpressions.FloatExpression expr, Void value) throws RuntimeException {
+  public Statistics<T> visitFloatConstant(ValueExpressions.FloatExpression expr, Void value) throws RuntimeException {
     return getStatistics(expr.getFloat());
   }
 
   @Override
-  public Statistics visitDoubleConstant(ValueExpressions.DoubleExpression expr, Void value) throws RuntimeException {
+  public Statistics<T> visitDoubleConstant(ValueExpressions.DoubleExpression expr, Void value) throws RuntimeException {
     return getStatistics(expr.getDouble());
   }
 
   @Override
-  public Statistics visitDateConstant(ValueExpressions.DateExpression expr, Void value) throws RuntimeException {
+  public Statistics<T> visitDateConstant(ValueExpressions.DateExpression expr, Void value) throws RuntimeException {
     long dateInMillis = expr.getDate();
     return getStatistics(dateInMillis);
   }
 
   @Override
-  public Statistics visitTimeStampConstant(ValueExpressions.TimeStampExpression tsExpr, Void value) throws RuntimeException {
+  public Statistics<T> visitTimeStampConstant(ValueExpressions.TimeStampExpression tsExpr, Void value) throws RuntimeException {
     long tsInMillis = tsExpr.getTimeStamp();
     return getStatistics(tsInMillis);
   }
 
   @Override
-  public Statistics visitTimeConstant(ValueExpressions.TimeExpression timeExpr, Void value) throws RuntimeException {
+  public Statistics<T> visitTimeConstant(ValueExpressions.TimeExpression timeExpr, Void value) throws RuntimeException {
     int milliSeconds = timeExpr.getTime();
     return getStatistics(milliSeconds);
   }
 
   @Override
-  public Statistics visitFunctionHolderExpression(FunctionHolderExpression holderExpr, Void value) throws RuntimeException {
+  public Statistics<T> visitFunctionHolderExpression(FunctionHolderExpression holderExpr, Void value) throws RuntimeException {
     FuncHolder funcHolder = holderExpr.getHolder();
 
     if (! (funcHolder instanceof DrillSimpleFuncHolder)) {
@@ -151,57 +154,62 @@ public Statistics visitFunctionHolderExpression(FunctionHolderExpression holderE
     return null;
   }
 
-  private IntStatistics getStatistics(int value) {
+  private Statistics<T> getStatistics(int value) {
     return getStatistics(value, value);
   }
 
-  private IntStatistics getStatistics(int min, int max) {
-    final IntStatistics intStatistics = new IntStatistics();
-    intStatistics.setMinMax(min, max);
-    return intStatistics;
+  @SuppressWarnings("unchecked")
+  private Statistics<T> getStatistics(int min, int max) {
+    final Statistics<T> statistics = Statistics.getStatsBasedOnType(PrimitiveType.PrimitiveTypeName.INT32);
+    ((IntStatistics)statistics).setMinMax(min, max);
+    return statistics;
   }
 
-  private BooleanStatistics getStatistics(boolean value) {
+  private Statistics<T> 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;
+  @SuppressWarnings("unchecked")
+  private Statistics<T> getStatistics(boolean min, boolean max) {
+    Statistics<T> statistics = Statistics.getStatsBasedOnType(PrimitiveType.PrimitiveTypeName.BOOLEAN);
+    ((BooleanStatistics)statistics).setMinMax(min, max);
+    return statistics;
   }
 
-  private LongStatistics getStatistics(long value) {
+  private Statistics<T> getStatistics(long value) {
     return getStatistics(value, value);
   }
 
-  private LongStatistics getStatistics(long min, long max) {
-    final LongStatistics longStatistics = new LongStatistics();
-    longStatistics.setMinMax(min, max);
-    return longStatistics;
+  @SuppressWarnings("unchecked")
+  private Statistics<T> getStatistics(long min, long max) {
+    final Statistics statistics = Statistics.getStatsBasedOnType(PrimitiveType.PrimitiveTypeName.INT64);
+    ((LongStatistics)statistics).setMinMax(min, max);
+    return statistics;
   }
 
-  private DoubleStatistics getStatistics(double value) {
+  private Statistics<T> getStatistics(double value) {
     return getStatistics(value, value);
   }
 
-  private DoubleStatistics getStatistics(double min, double max) {
-    final DoubleStatistics doubleStatistics = new DoubleStatistics();
-    doubleStatistics.setMinMax(min, max);
-    return doubleStatistics;
+  @SuppressWarnings("unchecked")
+  private Statistics<T> getStatistics(double min, double max) {
+    final Statistics<T> statistics = Statistics.getStatsBasedOnType(PrimitiveType.PrimitiveTypeName.DOUBLE);
+    ((DoubleStatistics)statistics).setMinMax(min, max);
+    return statistics;
   }
 
-  private FloatStatistics getStatistics(float value) {
+  private Statistics<T> getStatistics(float value) {
     return getStatistics(value, value);
   }
 
-  private FloatStatistics getStatistics(float min, float max) {
-    final FloatStatistics floatStatistics = new FloatStatistics();
-    floatStatistics.setMinMax(min, max);
-    return floatStatistics;
+  @SuppressWarnings("unchecked")
+  private Statistics<T> getStatistics(float min, float max) {
+    final Statistics<T> statistics = Statistics.getStatsBasedOnType(PrimitiveType.PrimitiveTypeName.FLOAT);
+    ((FloatStatistics)statistics).setMinMax(min, max);
+    return statistics;
   }
 
-  private Statistics evalCastFunc(FunctionHolderExpression holderExpr, Statistics input) {
+  private Statistics<T> evalCastFunc(FunctionHolderExpression holderExpr, Statistics input) {
     try {
       DrillSimpleFuncHolder funcHolder = (DrillSimpleFuncHolder) holderExpr.getHolder();
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorMetricRegistry.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorMetricRegistry.java
index d731ca4c24..dcb944512c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorMetricRegistry.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorMetricRegistry.java
@@ -29,7 +29,7 @@
 import org.apache.drill.exec.physical.impl.unorderedreceiver.UnorderedReceiverBatch;
 import org.apache.drill.exec.physical.impl.xsort.ExternalSortBatch;
 import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
-import org.apache.drill.exec.record.JoinBatchMemoryManager;
+import org.apache.drill.exec.record.AbstractBinaryRecordBatch;
 import org.apache.drill.exec.store.parquet.columnreaders.ParquetRecordReader;
 
 /**
@@ -53,9 +53,10 @@
     register(CoreOperatorType.EXTERNAL_SORT_VALUE, ExternalSortBatch.Metric.class);
     register(CoreOperatorType.PARQUET_ROW_GROUP_SCAN_VALUE, ParquetRecordReader.Metric.class);
     register(CoreOperatorType.FLATTEN_VALUE, FlattenRecordBatch.Metric.class);
-    register(CoreOperatorType.MERGE_JOIN_VALUE, JoinBatchMemoryManager.Metric.class);
-    register(CoreOperatorType.LATERAL_JOIN_VALUE, JoinBatchMemoryManager.Metric.class);
+    register(CoreOperatorType.MERGE_JOIN_VALUE, AbstractBinaryRecordBatch.Metric.class);
+    register(CoreOperatorType.LATERAL_JOIN_VALUE, AbstractBinaryRecordBatch.Metric.class);
     register(CoreOperatorType.UNNEST_VALUE, UnnestRecordBatch.Metric.class);
+    register(CoreOperatorType.UNION_VALUE, AbstractBinaryRecordBatch.Metric.class);
   }
 
   private static void register(final int operatorType, final Class<? extends MetricDef> metricDef) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
index eb32bc650f..07742f2e82 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
@@ -163,10 +163,10 @@ public SchemaPlus getRootSchema(final String userName) {
   }
 
   /**
-   *  Create and return a SchemaTree with given <i>schemaConfig</i> but some schemas (from storage plugins)
+   *  Create and return a {@link org.apache.calcite.schema.SchemaPlus} with given <i>schemaConfig</i> but some schemas (from storage plugins)
    *  could be initialized later.
    * @param schemaConfig
-   * @return
+   * @return A {@link org.apache.calcite.schema.SchemaPlus} with given <i>schemaConfig</i>.
    */
   public SchemaPlus getRootSchema(SchemaConfig schemaConfig) {
     return schemaTreeProvider.createRootSchema(schemaConfig);
@@ -174,7 +174,7 @@ public SchemaPlus getRootSchema(SchemaConfig schemaConfig) {
   /**
    *  Create and return a fully initialized SchemaTree with given <i>schemaConfig</i>.
    * @param schemaConfig
-   * @return
+   * @return A fully initialized SchemaTree with given <i>schemaConfig</i>.
    */
 
   public SchemaPlus getFullRootSchema(SchemaConfig schemaConfig) {
@@ -182,7 +182,7 @@ public SchemaPlus getFullRootSchema(SchemaConfig schemaConfig) {
   }
   /**
    * Get the user name of the user who issued the query that is managed by this QueryContext.
-   * @return
+   * @return The user name of the user who issued the query that is managed by this QueryContext.
    */
   @Override
   public String getQueryUserName() {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
index 36e74a0942..e7518b8ba0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
@@ -17,8 +17,6 @@
  */
 package org.apache.drill.exec.opt;
 
-import com.google.common.collect.Lists;
-
 import org.apache.drill.common.JSONOptions;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
@@ -28,9 +26,7 @@
 import org.apache.drill.common.logical.data.Filter;
 import org.apache.drill.common.logical.data.GroupingAggregate;
 import org.apache.drill.common.logical.data.Join;
-import org.apache.drill.common.logical.data.JoinCondition;
 import org.apache.drill.common.logical.data.LogicalOperator;
-import org.apache.drill.common.logical.data.NamedExpression;
 import org.apache.drill.common.logical.data.Order;
 import org.apache.drill.common.logical.data.Order.Ordering;
 import org.apache.drill.common.logical.data.Project;
@@ -53,6 +49,7 @@
 import org.apache.drill.exec.physical.config.SelectionVectorRemover;
 import org.apache.drill.exec.physical.config.Sort;
 import org.apache.drill.exec.physical.config.StreamingAggregate;
+import org.apache.drill.exec.physical.config.UnnestPOP;
 import org.apache.drill.exec.physical.config.WindowPOP;
 import org.apache.drill.exec.rpc.UserClientConnection;
 import org.apache.drill.exec.server.options.OptionManager;
@@ -64,6 +61,7 @@
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
+import java.util.stream.Collectors;
 
 public class BasicOptimizer extends Optimizer {
 //  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BasicOptimizer.class);
@@ -99,8 +97,7 @@ public PhysicalPlan optimize(final OptimizationContext context, final LogicalPla
         .version(logicalProperties.version)
         .generator(logicalProperties.generator)
         .options(new JSONOptions(context.getOptions().getOptionList())).build();
-    final PhysicalPlan p = new PhysicalPlan(props, physOps);
-    return p;
+    return new PhysicalPlan(props, physOps);
   }
 
   public static class BasicOptimizationContext implements OptimizationContext {
@@ -128,30 +125,28 @@ public OptionManager getOptions() {
      */
     private final LogicalPlan logicalPlan;
 
-    public LogicalConverter(final LogicalPlan logicalPlan) {
+    LogicalConverter(final LogicalPlan logicalPlan) {
       this.logicalPlan = logicalPlan;
     }
 
     @Override
     public PhysicalOperator visitGroupingAggregate(GroupingAggregate groupBy, Object value) throws OptimizerException {
-      final List<Ordering> orderDefs = Lists.newArrayList();
       PhysicalOperator input = groupBy.getInput().accept(this, value);
 
       if (groupBy.getKeys().size() > 0) {
-        for(NamedExpression e : groupBy.getKeys()) {
-          orderDefs.add(new Ordering(Direction.ASCENDING, e.getExpr(), NullDirection.FIRST));
-        }
+        List<Ordering> orderDefs = groupBy.getKeys().stream()
+            .map(e -> new Ordering(Direction.ASCENDING, e.getExpr(), NullDirection.FIRST))
+            .collect(Collectors.toList());
         input = new Sort(input, orderDefs, false);
       }
 
-      final StreamingAggregate sa = new StreamingAggregate(input, groupBy.getKeys(), groupBy.getExprs(), 1.0f);
-      return sa;
+      return new StreamingAggregate(input, groupBy.getKeys(), groupBy.getExprs(), 1.0f);
     }
 
     @Override
     public PhysicalOperator visitWindow(final Window window, final Object value) throws OptimizerException {
       PhysicalOperator input = window.getInput().accept(this, value);
-      final List<Ordering> ods = Lists.newArrayList();
+      final List<Ordering> ods = new ArrayList<>();
 
       input = new Sort(input, ods, false);
 
@@ -162,11 +157,7 @@ public PhysicalOperator visitWindow(final Window window, final Object value) thr
     @Override
     public PhysicalOperator visitOrder(final Order order, final Object value) throws OptimizerException {
       final PhysicalOperator input = order.getInput().accept(this, value);
-      final List<Ordering> ods = Lists.newArrayList();
-      for (Ordering o : order.getOrderings()){
-        ods.add(o);
-      }
-
+      final List<Ordering> ods = new ArrayList<>(order.getOrderings());
       return new SelectionVectorRemover(new Sort(input, ods, false));
     }
 
@@ -180,18 +171,20 @@ public PhysicalOperator visitLimit(final org.apache.drill.common.logical.data.Li
     @Override
     public PhysicalOperator visitJoin(final Join join, final Object value) throws OptimizerException {
       PhysicalOperator leftOp = join.getLeft().accept(this, value);
-      final List<Ordering> leftOrderDefs = Lists.newArrayList();
-      for(JoinCondition jc : join.getConditions()){
-        leftOrderDefs.add(new Ordering(Direction.ASCENDING, jc.getLeft()));
-      }
+
+      List<Ordering> leftOrderDefs = join.getConditions().stream()
+          .map(jc -> new Ordering(Direction.ASCENDING, jc.getLeft()))
+          .collect(Collectors.toList());
+
       leftOp = new Sort(leftOp, leftOrderDefs, false);
       leftOp = new SelectionVectorRemover(leftOp);
 
       PhysicalOperator rightOp = join.getRight().accept(this, value);
-      final List<Ordering> rightOrderDefs = Lists.newArrayList();
-      for(JoinCondition jc : join.getConditions()){
-        rightOrderDefs.add(new Ordering(Direction.ASCENDING, jc.getRight()));
-      }
+
+      List<Ordering> rightOrderDefs = join.getConditions().stream()
+          .map(jc -> new Ordering(Direction.ASCENDING, jc.getRight()))
+          .collect(Collectors.toList());
+
       rightOp = new Sort(rightOp, rightOrderDefs, false);
       rightOp = new SelectionVectorRemover(rightOp);
 
@@ -210,7 +203,7 @@ public PhysicalOperator visitScan(final Scan scan, final Object obj) throws Opti
       try {
         final StoragePlugin storagePlugin = queryContext.getStorage().getPlugin(config);
         final String user = userSession.getSession().getCredentials().getUserName();
-        return storagePlugin.getPhysicalScan(user, scan.getSelection());
+        return storagePlugin.getPhysicalScan(user, scan.getSelection(), userSession.getSession().getOptions());
       } catch (IOException | ExecutionSetupException e) {
         throw new OptimizerException("Failure while attempting to retrieve storage engine.", e);
       }
@@ -241,8 +234,8 @@ public PhysicalOperator visitFilter(final Filter filter, final Object obj) throw
     }
 
     @Override
-    public PhysicalOperator visitUnnest(final Unnest unnest, final Object obj) throws OptimizerException {
-      return new org.apache.drill.exec.physical.config.UnnestPOP(null, unnest.getColumn());
+    public PhysicalOperator visitUnnest(final Unnest unnest, final Object obj) {
+      return new UnnestPOP(null, unnest.getColumn());
     }
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractExchange.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractExchange.java
index 7885161451..96bdbaa340 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractExchange.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractExchange.java
@@ -55,7 +55,7 @@ public boolean isExecutable() {
   /**
    * Default sender parallelization width range is [1, Integer.MAX_VALUE] and no endpoint affinity
    * @param receiverFragmentEndpoints Endpoints assigned to receiver fragment if available, otherwise an empty list.
-   * @return
+   * @return Sender {@link org.apache.drill.exec.planner.fragment.ParallelizationInfo}.
    */
   @Override
   public ParallelizationInfo getSenderParallelizationInfo(List<DrillbitEndpoint> receiverFragmentEndpoints) {
@@ -66,7 +66,7 @@ public ParallelizationInfo getSenderParallelizationInfo(List<DrillbitEndpoint> r
    * Default receiver parallelization width range is [1, Integer.MAX_VALUE] and affinity to nodes where sender
    * fragments are running.
    * @param senderFragmentEndpoints Endpoints assigned to receiver fragment if available, otherwise an empty list.
-   * @return
+   * @return Receiver {@link org.apache.drill.exec.planner.fragment.ParallelizationInfo}.
    */
   @Override
   public ParallelizationInfo getReceiverParallelizationInfo(List<DrillbitEndpoint> senderFragmentEndpoints) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractSingle.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractSingle.java
index 1021465c35..9f3654e8ee 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractSingle.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractSingle.java
@@ -26,7 +26,6 @@
 
 /**
  * Describes an operator that expects a single child operator as its input.
- * @param <T> The type of Exec model supported.
  */
 public abstract class AbstractSingle extends AbstractBase {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractSingle.class);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Exchange.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Exchange.java
index 0aa3b70abb..56e9b9c90e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Exchange.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Exchange.java
@@ -83,7 +83,7 @@
    * max width and affinity to Drillbits.
    *
    * @param receiverFragmentEndpoints Endpoints assigned to receiver fragment if available, otherwise an empty list.
-   * @return
+   * @return Sender {@link org.apache.drill.exec.planner.fragment.ParallelizationInfo}.
    */
   @JsonIgnore
   ParallelizationInfo getSenderParallelizationInfo(List<DrillbitEndpoint> receiverFragmentEndpoints);
@@ -93,7 +93,7 @@
    * max width and affinity to Drillbits.
    *
    * @param senderFragmentEndpoints Endpoints assigned to receiver fragment if available, otherwise an empty list
-   * @return
+   * @return Receiver {@link org.apache.drill.exec.planner.fragment.ParallelizationInfo}.
    */
   @JsonIgnore
   ParallelizationInfo getReceiverParallelizationInfo(List<DrillbitEndpoint> senderFragmentEndpoints);
@@ -101,7 +101,7 @@
   /**
    * Return the feeding child of this operator node.
    *
-   * @return
+   * @return The feeding child of this operator node.
    */
   PhysicalOperator getChild();
 
@@ -110,4 +110,4 @@
    */
   @JsonIgnore
   ParallelizationDependency getParallelizationDependency();
-}
\ No newline at end of file
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/LateralContract.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/LateralContract.java
index 52e601e6f0..2f6d20dd7a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/LateralContract.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/LateralContract.java
@@ -27,19 +27,20 @@
 public interface LateralContract {
 
   /**
-   * Get reference to left side incoming of LateralJoinRecordBatch
-   * @return
+   * Get reference to left side incoming of {@link org.apache.drill.exec.physical.impl.join.LateralJoinBatch}.
+   * @return The incoming {@link org.apache.drill.exec.record.RecordBatch}
    */
   RecordBatch getIncoming();
 
   /**
-   * Get current record index in incoming to be processed
-   * @return
+   * Get current record index in incoming to be processed.
+   * @return The current record index in incoming to be processed.
    */
   int getRecordIndex();
 
   /**
-   * Get the current outcome of left incoming batch
+   * Get the current outcome of left incoming batch.
+   * @return The current outcome of left incoming batch.
    */
   IterOutcome getLeftOutcome();
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperator.java
index 35138c85e7..82fb53bf4a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperator.java
@@ -43,16 +43,13 @@
    * Describes whether or not a particular physical operator can actually be executed. Most physical operators can be
    * executed. However, Exchange nodes cannot be executed. In order to be executed, they must be converted into their
    * Exec sub components.
-   *
-   * @return
    */
   @JsonIgnore
   boolean isExecutable();
 
   /**
    * Describes the SelectionVector Mode for the output steam from this physical op.
-   * This property is used during physical plan creating using {@link PhysicalPlanCreator}.
-   * @return
+   * This property is used during physical plan creating using {@link org.apache.drill.exec.planner.physical.PhysicalPlanCreator}.
    */
   @JsonIgnore
   SelectionVectorMode getSVMode();
@@ -61,14 +58,12 @@
    * Provides capability to build a set of output based on traversing a query graph tree.
    *
    * @param physicalVisitor
-   * @return
    */
   <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E;
 
   /**
    * Regenerate with this node with a new set of children.  This is used in the case of materialization or optimization.
    * @param children
-   * @return
    */
   @JsonIgnore
   PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) throws ExecutionSetupException;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/ScanStats.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/ScanStats.java
index f2974e16c5..721f723ecb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/ScanStats.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/ScanStats.java
@@ -23,27 +23,27 @@
 
   public static final ScanStats ZERO_RECORD_TABLE = new ScanStats(GroupScanProperty.EXACT_ROW_COUNT, 0, 1, 1);
 
-  private final long recordCount;
-  private final float cpuCost;
-  private final float diskCost;
+  private final double recordCount;
+  private final double cpuCost;
+  private final double diskCost;
   private final GroupScanProperty property;
 
-  public ScanStats(GroupScanProperty property, long recordCount, float cpuCost, float diskCost) {
+  public ScanStats(GroupScanProperty property, double recordCount, double cpuCost, double diskCost) {
     this.recordCount = recordCount;
     this.cpuCost = cpuCost;
     this.diskCost = diskCost;
     this.property = property;
   }
 
-  public long getRecordCount() {
+  public double getRecordCount() {
     return recordCount;
   }
 
-  public float getCpuCost() {
+  public double getCpuCost() {
     return cpuCost;
   }
 
-  public float getDiskCost() {
+  public double getDiskCost() {
     return diskCost;
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Sender.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Sender.java
index c39cc9b0c6..340cbcef67 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Sender.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Sender.java
@@ -38,7 +38,7 @@
 
   /**
    * Get the receiver major fragment id that is opposite this sender.
-   * @return
+   * @return The receiver major fragment id that is opposite this sender.
    */
   @JsonProperty("receiver-major-fragment")
   public int getOppositeMajorFragmentId();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Store.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Store.java
index ffec99e74d..24338e0bff 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Store.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Store.java
@@ -61,14 +61,14 @@ public abstract Store getSpecificStore(PhysicalOperator child, int minorFragment
    * maxWidth value of 1 will be returned. In the case that there is no limit for parallelization, this method should
    * return Integer.MAX_VALUE.
    *
-   * @return
+   * @return The maximum allowable width for the Store operation.
    */
   @JsonIgnore
   public abstract int getMaxWidth();
 
   /**
    * Get the child of this store operator as this will be needed for parallelization materialization purposes.
-   * @return
+   * @return The child of this store operator.
    */
   public abstract PhysicalOperator getChild();
-}
\ No newline at end of file
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/LateralJoinPOP.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/LateralJoinPOP.java
index fab89a24e4..55ede96282 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/LateralJoinPOP.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/LateralJoinPOP.java
@@ -23,6 +23,7 @@
 import com.fasterxml.jackson.annotation.JsonTypeName;
 import com.google.common.base.Preconditions;
 import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.physical.base.AbstractJoinPop;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.PhysicalVisitor;
@@ -34,6 +35,9 @@
 public class LateralJoinPOP extends AbstractJoinPop {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(LateralJoinPOP.class);
 
+  @JsonProperty("excludedColumns")
+  private List<SchemaPath> excludedColumns;
+
   @JsonProperty("unnestForLateralJoin")
   private UnnestPOP unnestForLateralJoin;
 
@@ -41,15 +45,21 @@
   public LateralJoinPOP(
       @JsonProperty("left") PhysicalOperator left,
       @JsonProperty("right") PhysicalOperator right,
-      @JsonProperty("joinType") JoinRelType joinType) {
+      @JsonProperty("joinType") JoinRelType joinType,
+      @JsonProperty("excludedColumns") List<SchemaPath> excludedColumns) {
     super(left, right, joinType, null, null);
+    Preconditions.checkArgument(joinType != JoinRelType.FULL,
+      "Full outer join is currently not supported with Lateral Join");
+    Preconditions.checkArgument(joinType != JoinRelType.RIGHT,
+      "Right join is currently not supported with Lateral Join");
+    this.excludedColumns = excludedColumns;
   }
 
   @Override
   public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
     Preconditions.checkArgument(children.size() == 2,
       "Lateral join should have two physical operators");
-    LateralJoinPOP newPOP =  new LateralJoinPOP(children.get(0), children.get(1), joinType);
+    LateralJoinPOP newPOP =  new LateralJoinPOP(children.get(0), children.get(1), joinType, this.excludedColumns);
     newPOP.unnestForLateralJoin = this.unnestForLateralJoin;
     return newPOP;
   }
@@ -59,6 +69,11 @@ public UnnestPOP getUnnestForLateralJoin() {
     return this.unnestForLateralJoin;
   }
 
+  @JsonProperty("excludedColumns")
+  public List<SchemaPath> getExcludedColumns() {
+    return this.excludedColumns;
+  }
+
   public void setUnnestForLateralJoin(UnnestPOP unnest) {
     this.unnestForLateralJoin = unnest;
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
index 4a62752867..09e785e259 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
@@ -47,6 +47,8 @@
 import org.apache.drill.exec.testing.ControlsInjector;
 import org.apache.drill.exec.testing.ControlsInjectorFactory;
 import org.apache.drill.exec.util.CallBack;
+import org.apache.drill.exec.util.record.RecordBatchStats;
+import org.apache.drill.exec.util.record.RecordBatchStats.RecordBatchStatsContext;
 import org.apache.drill.exec.vector.AllocationHelper;
 import org.apache.drill.exec.vector.NullableVarCharVector;
 import org.apache.drill.exec.vector.SchemaChangeCallBack;
@@ -80,6 +82,8 @@
   private final BufferAllocator allocator;
   private final List<Map<String, String>> implicitColumnList;
   private String currentReaderClassName;
+  private final RecordBatchStatsContext batchStatsLogging;
+
   /**
    *
    * @param context
@@ -117,6 +121,7 @@ public ScanBatch(FragmentContext context,
       this.implicitColumnList = implicitColumnList;
       addImplicitVectors();
       currentReader = null;
+      batchStatsLogging = new RecordBatchStatsContext(context, oContext);
     } finally {
       oContext.getStats().stopProcessing();
     }
@@ -174,6 +179,7 @@ public IterOutcome next() {
         boolean isNewSchema = mutator.isNewSchema();
         populateImplicitVectorsAndSetCount();
         oContext.getStats().batchReceived(0, recordCount, isNewSchema);
+        logRecordBatchStats();
 
         if (recordCount == 0) {
           currentReader.close();
@@ -291,6 +297,45 @@ public TypedFieldId getValueVectorId(SchemaPath path) {
     return container.getValueAccessorById(clazz, ids);
   }
 
+  private void logRecordBatchStats() {
+    final int MAX_FQN_LENGTH = 50;
+
+    if (recordCount == 0) {
+      return; // NOOP
+    }
+
+    RecordBatchStats.logRecordBatchStats(
+      batchStatsLogging.getContextOperatorId(),
+      getFQNForLogging(MAX_FQN_LENGTH),
+      this,
+      batchStatsLogging,
+      logger);
+  }
+
+  /** Might truncate the FQN if too long */
+  private String getFQNForLogging(int maxLength) {
+    final String FQNKey = "FQN";
+    final ValueVector v = mutator.implicitFieldVectorMap.get(FQNKey);
+
+    final Object fqnObj;
+
+    if (v == null
+     || v.getAccessor().getValueCount() == 0
+     || (fqnObj = ((NullableVarCharVector) v).getAccessor().getObject(0)) == null) {
+
+      return "NA";
+    }
+
+    String fqn = fqnObj.toString();
+
+    if (fqn != null && fqn.length() > maxLength) {
+      fqn = fqn.substring(fqn.length() - maxLength, fqn.length());
+    }
+
+    return fqn;
+  }
+
+
   /**
    * Row set mutator implementation provided to record readers created by
    * this scan batch. Made visible so that tests can create this mutator
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/PriorityQueue.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/PriorityQueue.java
index e398f47485..3744b94200 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/PriorityQueue.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/PriorityQueue.java
@@ -66,6 +66,13 @@
    */
   SelectionVector4 getFinalSv4();
 
+  /**
+   * Cleanup the old state of queue and recreate a new one with HyperContainer containing vectors in input container
+   * and the corresponding indexes (in SV4 format) from input SelectionVector4
+   * @param container
+   * @param vector4
+   * @throws SchemaChangeException
+   */
   void resetQueue(VectorContainer container, SelectionVector4 vector4) throws SchemaChangeException;
 
   /**
@@ -73,5 +80,7 @@
    */
   void cleanup();
 
+  boolean isInitialized();
+
   TemplateClassDefinition<PriorityQueue> TEMPLATE_DEFINITION = new TemplateClassDefinition<>(PriorityQueue.class, PriorityQueueTemplate.class);
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/PriorityQueueTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/PriorityQueueTemplate.java
index 97e26b662c..e39dce3a10 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/PriorityQueueTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/PriorityQueueTemplate.java
@@ -40,10 +40,18 @@
 public abstract class PriorityQueueTemplate implements PriorityQueue {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PriorityQueueTemplate.class);
 
-  private SelectionVector4 heapSv4; //This holds the heap
+  // This holds the min heap of the record indexes. Heapify condition is based on actual record though. Only records
+  // meeting the heap condition have their indexes in this heap. Actual record are stored inside the hyperBatch. Since
+  // hyperBatch contains ValueVectors from all the incoming batches, the indexes here consider both BatchNumber and
+  // RecordNumber.
+  private SelectionVector4 heapSv4;
   private SelectionVector4 finalSv4; //This is for final sorted output
+
+  // This stores the actual incoming record batches
   private ExpandableHyperContainer hyperBatch;
   private BufferAllocator allocator;
+
+  // Limit determines the number of record to output and hold in queue.
   private int limit;
   private int queueSize = 0;
   private int batchCount = 0;
@@ -54,7 +62,11 @@ public void init(int limit, BufferAllocator allocator,  boolean hasSv2) throws S
     this.limit = limit;
     this.allocator = allocator;
     @SuppressWarnings("resource")
+    // It's allocating memory to store (limit+1) indexes. When first limit number of record indexes are stored then all
+    // the other record indexes are kept at (limit+1) and evaluated with the root element of heap to determine if
+    // this new element will reside in heap or not.
     final DrillBuf drillBuf = allocator.buffer(4 * (limit + 1));
+    // Heap is a SelectionVector4 since it stores indexes for record relative to their batches.
     heapSv4 = new SelectionVector4(drillBuf, limit, Character.MAX_VALUE);
     this.hasSv2 = hasSv2;
   }
@@ -103,11 +115,17 @@ public void add(RecordBatchData batch) throws SchemaChangeException{
     if (hasSv2) {
       sv2 = batch.getSv2();
     }
+    // Will only be called until queueSize has reached the limit which means it has seen limit number of records in
+    // one or many batches. For each new record siftUp (or heapify) to adjust min heap property is called.
     for (; queueSize < limit && count < batch.getRecordCount();  count++) {
       heapSv4.set(queueSize, batchCount, hasSv2 ? sv2.getIndex(count) : count);
       queueSize++;
       siftUp();
     }
+
+    // For all the other records which fall beyond limit, it compares them with the root element in the current heap
+    // and perform heapify if need be. Note: Even though heapSv4 stores only limit+1 indexes but in hyper batch we
+    // are still keeping all the records unless purge is called.
     for (; count < batch.getRecordCount(); count++) {
       heapSv4.set(limit, batchCount, hasSv2 ? sv2.getIndex(count) : count);
       if (compare(limit, 0) < 0) {
@@ -153,15 +171,35 @@ public SelectionVector4 getFinalSv4() {
   public void cleanup() {
     if (heapSv4 != null) {
       heapSv4.clear();
+      heapSv4 = null;
     }
     if (hyperBatch != null) {
       hyperBatch.clear();
+      hyperBatch = null;
     }
     if (finalSv4 != null) {
       finalSv4.clear();
+      finalSv4 = null;
     }
+    batchCount = 0;
   }
 
+  /**
+   * When cleanup is called then heapSv4 is cleared and set to null and is only initialized during init call. Hence
+   * this is used to determine if priority queue is initialized or not.
+   * @return - true - queue is still initialized
+   *           false - queue is not yet initialized and before using queue init should be called
+   */
+  public boolean isInitialized() {
+    return (heapSv4 != null);
+  }
+
+  /**
+   * Perform Heapify for the record stored at index which was added as leaf node in the array. The new record is
+   * compared with the record stored at parent index. Since the new record index will flow up in the array hence the
+   * name siftUp
+   * @throws SchemaChangeException
+   */
   private void siftUp() throws SchemaChangeException {
     int p = queueSize - 1;
     while (p > 0) {
@@ -174,6 +212,14 @@ private void siftUp() throws SchemaChangeException {
     }
   }
 
+  /**
+   * Compares the record stored at the index of 0th index element of heapSv4 (or root element) with the record
+   * stored at index of limit index element of heapSv4 (or new element). If the root element is greater than new element
+   * then new element is discarded else root element is replaced with new element and again heapify is performed on
+   * new root element.
+   * This is done for all the records which are seen after the queue is filled with limit number of record indexes.
+   * @throws SchemaChangeException
+   */
   private void siftDown() throws SchemaChangeException {
     int p = 0;
     int next;
@@ -196,6 +242,12 @@ private void siftDown() throws SchemaChangeException {
     }
   }
 
+  /**
+   * Pop the root element which holds the minimum value in heap. In this case root element will be the index of
+   * record with minimum value. After extracting the root element it swaps the root element with last element in
+   * heapSv4 and does heapify (by calling siftDown) again.
+   * @return - Index for
+   */
   public int pop() {
     int value = heapSv4.get(0);
     swap(0, queueSize - 1);
@@ -220,9 +272,25 @@ public int compare(int leftIndex, int rightIndex) throws SchemaChangeException {
     return doEval(sv1, sv2);
   }
 
+  /**
+   * Stores the reference to the hyperBatch container which holds all the records across incoming batches in it. This
+   * is used in doEval function to compare records in this hyper batch at given indexes.
+   * @param incoming - reference to hyperBatch
+   * @param outgoing - null
+   * @throws SchemaChangeException
+   */
   public abstract void doSetup(@Named("incoming") VectorContainer incoming,
                                @Named("outgoing") RecordBatch outgoing)
                        throws SchemaChangeException;
+
+  /**
+   * Evaluates the value of record at leftIndex and rightIndex w.r.t min heap condition. It is used in
+   * {@link PriorityQueueTemplate#compare(int, int)} method while Heapifying the queue.
+   * @param leftIndex
+   * @param rightIndex
+   * @return
+   * @throws SchemaChangeException
+   */
   public abstract int doEval(@Named("leftIndex") int leftIndex,
                              @Named("rightIndex") int rightIndex)
                       throws SchemaChangeException;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/TopNBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/TopNBatch.java
index 366e4e8888..4fc0d1596a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/TopNBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/TopNBatch.java
@@ -52,6 +52,7 @@
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.apache.drill.exec.record.ExpandableHyperContainer;
+import org.apache.drill.exec.record.HyperVectorWrapper;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.SchemaUtil;
 import org.apache.drill.exec.record.SimpleRecordBatch;
@@ -69,6 +70,16 @@
 import com.sun.codemodel.JConditional;
 import com.sun.codemodel.JExpr;
 
+import static org.apache.drill.exec.record.RecordBatch.IterOutcome.EMIT;
+import static org.apache.drill.exec.record.RecordBatch.IterOutcome.NONE;
+import static org.apache.drill.exec.record.RecordBatch.IterOutcome.OK;
+import static org.apache.drill.exec.record.RecordBatch.IterOutcome.OK_NEW_SCHEMA;
+
+/**
+ * Operator Batch which implements the TopN functionality. It is more efficient than (sort + limit) since unlike sort
+ * it doesn't have to store all the input data to sort it first and then apply limit on the sorted data. Instead
+ * internally it maintains a priority queue backed by a heap with the size being same as limit value.
+ */
 public class TopNBatch extends AbstractRecordBatch<TopN> {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TopNBatch.class);
 
@@ -84,12 +95,15 @@
   private boolean schemaChanged = false;
   private PriorityQueue priorityQueue;
   private TopN config;
-  SelectionVector4 sv4;
+  private SelectionVector4 sv4;
   private long countSincePurge;
   private int batchCount;
   private Copier copier;
   private boolean first = true;
   private int recordCount = 0;
+  private IterOutcome lastKnownOutcome = OK;
+  private boolean firstBatchForSchema = true;
+  private boolean hasOutputRecords = false;
 
   public TopNBatch(TopN popConfig, FragmentContext context, RecordBatch incoming) throws OutOfMemoryException {
     super(popConfig, context);
@@ -117,12 +131,7 @@ public SelectionVector4 getSelectionVector4() {
 
   @Override
   public void close() {
-    if (sv4 != null) {
-      sv4.clear();
-    }
-    if (priorityQueue != null) {
-      priorityQueue.cleanup();
-    }
+    releaseResource();
     super.close();
   }
 
@@ -134,6 +143,9 @@ public void buildSchema() throws SchemaChangeException {
       case OK:
       case OK_NEW_SCHEMA:
         for (VectorWrapper<?> w : incoming) {
+          // TODO: Not sure why the special handling for AbstractContainerVector is needed since creation of child
+          // vectors is taken care correctly if the field is retrieved from incoming vector and passed to it rather than
+          // creating a new Field instance just based on name and type.
           @SuppressWarnings("resource")
           ValueVector v = c.addOrGet(w.getField());
           if (v instanceof AbstractContainerVector) {
@@ -162,8 +174,11 @@ public void buildSchema() throws SchemaChangeException {
         return;
       case NONE:
         state = BatchState.DONE;
-      default:
         return;
+      case EMIT:
+        throw new IllegalStateException("Unexpected EMIT outcome received in buildSchema phase");
+      default:
+        throw new IllegalStateException("Unexpected outcome received in buildSchema phase");
     }
   }
 
@@ -171,47 +186,69 @@ public void buildSchema() throws SchemaChangeException {
   public IterOutcome innerNext() {
     recordCount = 0;
     if (state == BatchState.DONE) {
-      return IterOutcome.NONE;
+      return NONE;
     }
-    if (schema != null) {
-      if (getSelectionVector4().next()) {
-        recordCount = sv4.getCount();
-        return IterOutcome.OK;
-      } else {
-        recordCount = 0;
-        return IterOutcome.NONE;
-      }
+
+    // Check if anything is remaining from previous record boundary
+    if (hasOutputRecords) {
+      return handleRemainingOutput();
     }
 
+    // Reset the TopN state for next iteration
+    resetTopNState();
+
     try{
+      boolean incomingHasSv2 = false;
+      switch (incoming.getSchema().getSelectionVectorMode()) {
+        case NONE: {
+          break;
+        }
+        case TWO_BYTE: {
+          incomingHasSv2 = true;
+          break;
+        }
+        case FOUR_BYTE: {
+          throw new SchemaChangeException("TopN doesn't support incoming with SV4 mode");
+        }
+        default:
+          throw new UnsupportedOperationException("Unsupported SV mode detected in TopN incoming batch");
+      }
+
       outer: while (true) {
         Stopwatch watch = Stopwatch.createStarted();
-        IterOutcome upstream;
         if (first) {
-          upstream = IterOutcome.OK_NEW_SCHEMA;
+          lastKnownOutcome = IterOutcome.OK_NEW_SCHEMA;
+          // Create the SV4 object upfront to be used for both empty and non-empty incoming batches at EMIT boundary
+          sv4 = new SelectionVector4(context.getAllocator(), 0);
           first = false;
         } else {
-          upstream = next(incoming);
+          lastKnownOutcome = next(incoming);
         }
-        if (upstream == IterOutcome.OK && schema == null) {
-          upstream = IterOutcome.OK_NEW_SCHEMA;
+        if (lastKnownOutcome == OK && schema == null) {
+          lastKnownOutcome = IterOutcome.OK_NEW_SCHEMA;
           container.clear();
         }
         logger.debug("Took {} us to get next", watch.elapsed(TimeUnit.MICROSECONDS));
-        switch (upstream) {
+        switch (lastKnownOutcome) {
         case NONE:
           break outer;
         case NOT_YET:
           throw new UnsupportedOperationException();
         case OUT_OF_MEMORY:
         case STOP:
-          return upstream;
+          return lastKnownOutcome;
         case OK_NEW_SCHEMA:
           // only change in the case that the schema truly changes.  Artificial schema changes are ignored.
+          // schema change handling in case when EMIT is also seen is same as without EMIT. i.e. only if union type
+          // is enabled it will be handled.
+          container.clear();
+          firstBatchForSchema = true;
           if (!incoming.getSchema().equals(schema)) {
             if (schema != null) {
               if (!unionTypeEnabled) {
-                throw new UnsupportedOperationException("Sort doesn't currently support sorts with changing schemas.");
+                throw new UnsupportedOperationException(String.format("TopN currently doesn't support changing " +
+                  "schemas with union type disabled. Please try enabling union type: %s and re-execute the query",
+                  ExecConstants.ENABLE_UNION_TYPE_KEY));
               } else {
                 this.schema = SchemaUtil.mergeSchemas(this.schema, incoming.getSchema());
                 purgeAndResetPriorityQueue();
@@ -223,10 +260,15 @@ public IterOutcome innerNext() {
           }
           // fall through.
         case OK:
+        case EMIT:
           if (incoming.getRecordCount() == 0) {
             for (VectorWrapper<?> w : incoming) {
               w.clear();
             }
+            // Release memory for incoming SV2 vector
+            if (incomingHasSv2) {
+              incoming.getSelectionVector2().clear();
+            }
             break;
           }
           countSincePurge += incoming.getRecordCount();
@@ -240,10 +282,16 @@ public IterOutcome innerNext() {
           boolean success = false;
           try {
             if (priorityQueue == null) {
-              assert !schemaChanged;
               priorityQueue = createNewPriorityQueue(new ExpandableHyperContainer(batch.getContainer()), config.getLimit());
+            } else if (!priorityQueue.isInitialized()) {
+              // means priority queue is cleaned up after producing output for first record boundary. We should
+              // initialize it for next record boundary
+              priorityQueue.init(config.getLimit(), oContext.getAllocator(),
+                schema.getSelectionVectorMode() == SelectionVectorMode.TWO_BYTE);
             }
             priorityQueue.add(batch);
+            // Based on static threshold of number of batches, perform purge operation to release the memory for
+            // RecordBatches which are of no use or doesn't fall under TopN category
             if (countSincePurge > config.getLimit() && batchCount > batchPurgeThreshold) {
               purge();
               countSincePurge = 0;
@@ -259,25 +307,29 @@ public IterOutcome innerNext() {
         default:
           throw new UnsupportedOperationException();
         }
+
+        // If the last seen outcome is EMIT then break the loop. We do it here since we want to process the batch
+        // with records and EMIT outcome in above case statements
+        if (lastKnownOutcome == EMIT) {
+          break;
+        }
       }
 
-      if (schema == null || priorityQueue == null) {
+      // PriorityQueue can be null here if first batch is received with OK_NEW_SCHEMA and is empty and second next()
+      // call returned NONE or EMIT.
+      // PriorityQueue can be uninitialized here if only empty batch is received between 2 EMIT outcome.
+      if (schema == null || (priorityQueue == null || !priorityQueue.isInitialized())) {
         // builder may be null at this point if the first incoming batch is empty
-        state = BatchState.DONE;
-        return IterOutcome.NONE;
+        return handleEmptyBatches(lastKnownOutcome);
       }
 
       priorityQueue.generate();
+      prepareOutputContainer(priorityQueue.getHyperBatch(), priorityQueue.getFinalSv4());
 
-      this.sv4 = priorityQueue.getFinalSv4();
-      container.clear();
-      for (VectorWrapper<?> w : priorityQueue.getHyperBatch()) {
-        container.add(w.getValueVectors());
-      }
-      container.buildSchema(BatchSchema.SelectionVectorMode.FOUR_BYTE);
-      recordCount = sv4.getCount();
-      return IterOutcome.OK_NEW_SCHEMA;
-
+      // With EMIT outcome control will come here multiple times whereas without EMIT outcome control will only come
+      // here once. In EMIT outcome case if there is schema change in any iteration then that will be handled by
+      // lastKnownOutcome.
+      return getFinalOutcome();
     } catch(SchemaChangeException | ClassTransformationException | IOException ex) {
       kill(false);
       logger.error("Failure during query", ex);
@@ -286,14 +338,27 @@ public IterOutcome innerNext() {
     }
   }
 
+  /**
+   * When PriorityQueue is built up then it stores the list of limit number of record indexes (in heapSv4) which falls
+   * under TopN category. But it also stores all the incoming RecordBatches with all records inside a HyperContainer
+   * (hyperBatch). When a certain threshold of batches are reached then this method is called which copies the limit
+   * number of records whose indexes are stored in heapSv4 out of HyperBatch to a new VectorContainer and releases
+   * all other records and their batches. Later this new VectorContainer is stored inside the HyperBatch and it's
+   * corresponding indexes are stored in the heapSv4 vector. This is done to avoid holding up lot's of Record Batches
+   * which can create OutOfMemory condition.
+   * @throws SchemaChangeException
+   */
   private void purge() throws SchemaChangeException {
     Stopwatch watch = Stopwatch.createStarted();
     VectorContainer c = priorityQueue.getHyperBatch();
+
+    // Simple VectorConatiner which stores limit number of records only. The records whose indexes are stored inside
+    // selectionVector4 below are only copied from Hyper container to this simple container.
     VectorContainer newContainer = new VectorContainer(oContext);
     @SuppressWarnings("resource")
+    // SV4 storing the limit number of indexes
     SelectionVector4 selectionVector4 = priorityQueue.getSv4();
     SimpleSV4RecordBatch batch = new SimpleSV4RecordBatch(c, selectionVector4, context);
-    SimpleSV4RecordBatch newBatch = new SimpleSV4RecordBatch(newContainer, null, context);
     if (copier == null) {
       copier = GenericSV4Copier.createCopier(batch, newContainer, null);
     } else {
@@ -308,25 +373,14 @@ private void purge() throws SchemaChangeException {
     @SuppressWarnings("resource")
     SortRecordBatchBuilder builder = new SortRecordBatchBuilder(oContext.getAllocator());
     try {
-      do {
-        int count = selectionVector4.getCount();
-        int copiedRecords = copier.copyRecords(0, count);
-        assert copiedRecords == count;
-        for (VectorWrapper<?> v : newContainer) {
-          ValueVector.Mutator m = v.getValueVector().getMutator();
-          m.setValueCount(count);
-        }
-        newContainer.buildSchema(BatchSchema.SelectionVectorMode.NONE);
-        newContainer.setRecordCount(count);
-        builder.add(newBatch);
-      } while (selectionVector4.next());
-      selectionVector4.clear();
-      c.clear();
+      // Purge all the existing batches to a new batch which only holds the selected records
+      copyToPurge(newContainer, builder);
+      // New VectorContainer that contains only limit number of records and is later passed to resetQueue to create a
+      // HyperContainer backing the priority queue out of it
       VectorContainer newQueue = new VectorContainer();
       builder.build(newQueue);
       priorityQueue.resetQueue(newQueue, builder.getSv4().createNewWrapperCurrent());
       builder.getSv4().clear();
-      selectionVector4.clear();
     } finally {
       DrillAutoCloseables.closeNoChecked(builder);
     }
@@ -414,25 +468,12 @@ public void purgeAndResetPriorityQueue() throws SchemaChangeException, ClassTran
     @SuppressWarnings("resource")
     final SelectionVector4 selectionVector4 = priorityQueue.getSv4();
     final SimpleSV4RecordBatch batch = new SimpleSV4RecordBatch(c, selectionVector4, context);
-    final SimpleSV4RecordBatch newBatch = new SimpleSV4RecordBatch(newContainer, null, context);
     copier = GenericSV4Copier.createCopier(batch, newContainer, null);
     @SuppressWarnings("resource")
     SortRecordBatchBuilder builder = new SortRecordBatchBuilder(oContext.getAllocator());
     try {
-      do {
-        final int count = selectionVector4.getCount();
-        final int copiedRecords = copier.copyRecords(0, count);
-        assert copiedRecords == count;
-        for (VectorWrapper<?> v : newContainer) {
-          ValueVector.Mutator m = v.getValueVector().getMutator();
-          m.setValueCount(count);
-        }
-        newContainer.buildSchema(BatchSchema.SelectionVectorMode.NONE);
-        newContainer.setRecordCount(count);
-        builder.add(newBatch);
-      } while (selectionVector4.next());
-      selectionVector4.clear();
-      c.clear();
+      // Purge all the existing batches to a new batch which only holds the selected records
+      copyToPurge(newContainer, builder);
       final VectorContainer oldSchemaContainer = new VectorContainer(oContext);
       builder.build(oldSchemaContainer);
       oldSchemaContainer.setRecordCount(builder.getSv4().getCount());
@@ -458,6 +499,190 @@ protected void killIncoming(boolean sendUpstream) {
     incoming.kill(sendUpstream);
   }
 
+  /**
+   * Resets TopNBatch state to process next incoming batches independent of already seen incoming batches.
+   */
+  private void resetTopNState() {
+    lastKnownOutcome = OK;
+    countSincePurge = 0;
+    batchCount = 0;
+    hasOutputRecords = false;
+    releaseResource();
+  }
+
+  /**
+   * Cleanup resources held by TopN Batch such as sv4, priority queue and outgoing container
+   */
+  private void releaseResource() {
+    if (sv4 != null) {
+      sv4.clear();
+    }
+
+    if (priorityQueue != null) {
+      priorityQueue.cleanup();
+    }
+    container.zeroVectors();
+  }
+
+  /**
+   * Returns the final IterOutcome which TopN should return for this next call. Return OK_NEW_SCHEMA with first output
+   * batch after a new schema is seen. This is indicated by firstBatchSchema flag. It is also true for very first
+   * output batch after buildSchema()phase too since in buildSchema() a dummy schema was returned downstream without
+   * correct SelectionVectorMode.
+   * In other cases when there is no schema change then either OK or EMIT is returned with output batches depending upon
+   * if EMIT is seen or not. In cases when EMIT is not seen then OK is always returned with an output batch. When all
+   * the data is returned then NONE is sent in the end.
+   *
+   * @return - IterOutcome - outcome to send downstream
+   */
+  private IterOutcome getFinalOutcome() {
+    IterOutcome outcomeToReturn;
+
+    if (firstBatchForSchema) {
+      outcomeToReturn = OK_NEW_SCHEMA;
+      firstBatchForSchema = false;
+    } else if (recordCount == 0) {
+      // get the outcome to return before calling refresh since that resets the lastKnowOutcome to OK
+      outcomeToReturn = lastKnownOutcome == EMIT ? EMIT : NONE;
+      resetTopNState();
+    } else if (lastKnownOutcome == EMIT) {
+      // in case of EMIT check if this output batch returns all the data or not. If yes then return EMIT along with this
+      // output batch else return OK. Remaining data will be sent downstream in subsequent next() call.
+      final boolean hasMoreRecords = sv4.hasNext();
+      outcomeToReturn = (hasMoreRecords) ? OK : EMIT;
+      hasOutputRecords = hasMoreRecords;
+    } else {
+      outcomeToReturn = OK;
+    }
+
+    return outcomeToReturn;
+  }
+
+  /**
+   * Copies all the selected records into the new container to purge all the incoming batches into a single batch.
+   * @param newContainer - New container holding the ValueVectors with selected records
+   * @param batchBuilder - Builder to build hyper vectors batches
+   * @throws SchemaChangeException
+   */
+  private void copyToPurge(VectorContainer newContainer, SortRecordBatchBuilder batchBuilder)
+    throws SchemaChangeException {
+    final VectorContainer c = priorityQueue.getHyperBatch();
+    final SelectionVector4 queueSv4 = priorityQueue.getSv4();
+    final SimpleSV4RecordBatch newBatch = new SimpleSV4RecordBatch(newContainer, null, context);
+
+    do {
+      // count is the limit number of records required by TopN batch
+      final int count = queueSv4.getCount();
+      // Transfers count number of records from hyperBatch to simple container
+      final int copiedRecords = copier.copyRecords(0, count);
+      assert copiedRecords == count;
+      for (VectorWrapper<?> v : newContainer) {
+        ValueVector.Mutator m = v.getValueVector().getMutator();
+        m.setValueCount(count);
+      }
+      newContainer.buildSchema(BatchSchema.SelectionVectorMode.NONE);
+      newContainer.setRecordCount(count);
+      // Store all the batches containing limit number of records
+      batchBuilder.add(newBatch);
+    } while (queueSv4.next());
+    // Release the memory stored for the priority queue heap to store indexes
+    queueSv4.clear();
+    // Release the memory from HyperBatch container
+    c.clear();
+  }
+
+  /**
+   * Prepares an output container with batches from Priority Queue for each record boundary. In case when this is the
+   * first batch for the known schema (indicated by true value of firstBatchForSchema) the output container is cleared
+   * and recreated with new HyperVectorWrapper objects and ValueVectors from PriorityQueue. In cases when the schema
+   * has not changed then it prepares the container keeping the VectorWrapper and SV4 references as is since that is
+   * what is needed by downstream operator.
+   */
+  private void prepareOutputContainer(VectorContainer dataContainer, SelectionVector4 dataSv4) {
+    container.zeroVectors();
+    hasOutputRecords = true;
+    // Check if this is the first output batch for the new known schema. If yes then prepare the output container
+    // with the proper vectors, otherwise re-use the previous vectors.
+    if (firstBatchForSchema) {
+      container.clear();
+      for (VectorWrapper<?> w : dataContainer) {
+        container.add(w.getValueVectors());
+      }
+      container.buildSchema(BatchSchema.SelectionVectorMode.FOUR_BYTE);
+      sv4 = dataSv4;
+    } else {
+      // Schema didn't changed so we should keep the reference of HyperVectorWrapper in outgoing container intact and
+      // populate the HyperVectorWrapper with new list of vectors. Here the assumption is order of ValueVectors is same
+      // across multiple record boundary unless a new schema is observed
+      int index = 0;
+      for (VectorWrapper<?> w : dataContainer) {
+        HyperVectorWrapper wrapper = (HyperVectorWrapper<?>) container.getValueVector(index++);
+        wrapper.updateVectorList(w.getValueVectors());
+      }
+      // Since the reference of SV4 is held by downstream operator and there is no schema change, so just copy the
+      // underlying buffer from priority queue sv4.
+      this.sv4.copy(dataSv4);
+    }
+    recordCount = sv4.getCount();
+    container.setRecordCount(recordCount);
+  }
+
+  /**
+   * Method handles returning correct outcome and setting recordCount for output container when next() is called
+   * multiple time for single record boundary. It handles cases when some output was already returned at current record
+   * boundary but Either there is more left to return OR proper outcome with empty batch is left to return.
+   * Example: For first EMIT record boundary if all the records were returned in previous call with OK_NEW_SCHEMA
+   * outcome, then this method will handle returning empty output batch with EMIT outcome in subsequent next() call.
+   * @return - Outcome to return downstream
+   */
+  private IterOutcome handleRemainingOutput() {
+    // if priority queue is not null that means the incoming batches were non-empty. And if there are more records
+    // to send downstream for this record boundary
+    if (priorityQueue != null && sv4.next()) {
+      recordCount = sv4.getCount();
+      container.setRecordCount(recordCount);
+    } else { // This means that either:
+      // 1) Priority Queue was not null and all records have been sent downstream for this record boundary
+      // 2) Or Priority Queue is null, since all the incoming batches were empty for current record boundary (or EMIT
+      // outcome). In the previous call we must have returned OK_NEW_SCHEMA along with SV4 container, so it will
+      // return EMIT outcome now
+      recordCount = 0;
+      container.setRecordCount(0);
+    }
+    return getFinalOutcome();
+  }
+
+  /**
+   * Method to handle preparing output container and returning proper outcome to downstream when either NONE or only
+   * empty batches have been seen but with EMIT outcome. In either of the case PriorityQueue is not created yet since no
+   * actual records have been received so far.
+   * @param incomingOutcome - outcome received from upstream. Either NONE or EMIT
+   * @return - outcome to return downstream. NONE when incomingOutcome is NONE. OK_NEW_SCHEMA/EMIT when incomingOutcome
+   * is EMIT and is first/non-first empty input batch respectively.
+   */
+  private IterOutcome handleEmptyBatches(IterOutcome incomingOutcome) {
+    IterOutcome outcomeToReturn = incomingOutcome;
+
+    // In case of NONE it will change state to DONE and return NONE whereas in case of
+    // EMIT it has to still continue working for future records.
+    if (incomingOutcome == NONE) { // this means we saw NONE
+      state = BatchState.DONE;
+      container.clear();
+      recordCount = 0;
+      container.setRecordCount(recordCount);
+    } else if (incomingOutcome == EMIT) {
+      // since priority queue is null that means it has not seen any batch with data
+      assert (countSincePurge == 0 && batchCount == 0);
+      final VectorContainer hyperContainer = new ExpandableHyperContainer(incoming.getContainer());
+      prepareOutputContainer(hyperContainer, sv4);
+
+      // update the outcome to return
+      outcomeToReturn = getFinalOutcome();
+    }
+
+    return outcomeToReturn;
+  }
+
   public static class SimpleSV4RecordBatch extends SimpleRecordBatch {
     private SelectionVector4 sv4;
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WriterRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WriterRecordBatch.java
index b58cb55552..65d0c54d1c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WriterRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WriterRecordBatch.java
@@ -95,8 +95,14 @@ public IterOutcome innerNext() {
             return upstream;
 
           case NOT_YET:
-          case NONE:
             break;
+          case NONE:
+            if (schema != null) {
+              // Schema is for the output batch schema which is setup in setupNewSchema(). Since the output
+              // schema is fixed ((Fragment(VARCHAR), Number of records written (BIGINT)) we should set it
+              // up even with 0 records for it to be reported back to the client.
+              break;
+            }
 
           case OK_NEW_SCHEMA:
             setupNewSchema();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggBatch.java
index 47f1017b34..d37631be45 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggBatch.java
@@ -19,16 +19,19 @@
 
 import java.io.IOException;
 import java.util.List;
+import java.util.Map;
 
 import com.google.common.collect.Lists;
-import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.expression.ErrorCollector;
 import org.apache.drill.common.expression.ErrorCollectorImpl;
+import org.apache.drill.common.expression.FunctionCall;
 import org.apache.drill.common.expression.FunctionHolderExpression;
 import org.apache.drill.common.expression.IfExpression;
 import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.logical.data.NamedExpression;
+import org.apache.drill.common.map.CaseInsensitiveMap;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.compile.sig.GeneratorMapping;
 import org.apache.drill.exec.compile.sig.MappingSet;
@@ -50,11 +53,14 @@
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.RecordBatchMemoryManager;
+import org.apache.drill.exec.record.RecordBatchSizer;
 import org.apache.drill.exec.record.TypedFieldId;
 import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.record.selection.SelectionVector2;
 import org.apache.drill.exec.record.selection.SelectionVector4;
 import org.apache.drill.exec.vector.AllocationHelper;
+import org.apache.drill.exec.vector.FixedWidthVector;
 import org.apache.drill.exec.vector.ValueVector;
 
 import com.sun.codemodel.JExpr;
@@ -72,6 +78,12 @@
   private BatchSchema incomingSchema;
   private boolean wasKilled;
 
+  private int numGroupByExprs, numAggrExprs;
+
+  // This map saves the mapping between outgoing column and incoming column.
+  private Map<String, String> columnMapping;
+  private final HashAggMemoryManager hashAggMemoryManager;
+
   private final GeneratorMapping UPDATE_AGGR_INSIDE =
       GeneratorMapping.create("setupInterior" /* setup method */, "updateAggrValuesInternal" /* eval method */,
           "resetValues" /* reset */, "cleanup" /* cleanup */);
@@ -85,8 +97,69 @@
           "htRowIdx" /* workspace index */, "incoming" /* read container */, "outgoing" /* write container */,
           "aggrValuesContainer" /* workspace container */, UPDATE_AGGR_INSIDE, UPDATE_AGGR_OUTSIDE, UPDATE_AGGR_INSIDE);
 
+  public int getOutputRowCount() {
+    return hashAggMemoryManager.getOutputRowCount();
+  }
+
+  public RecordBatchMemoryManager getRecordBatchMemoryManager() {
+    return hashAggMemoryManager;
+  }
+
+  private class HashAggMemoryManager extends RecordBatchMemoryManager {
+    private int valuesRowWidth = 0;
+
+    HashAggMemoryManager(int outputBatchSize) {
+      super(outputBatchSize);
+    }
 
-  public HashAggBatch(HashAggregate popConfig, RecordBatch incoming, FragmentContext context) throws ExecutionSetupException {
+    @Override
+    public void update() {
+      // Get sizing information for the batch.
+      setRecordBatchSizer(new RecordBatchSizer(incoming));
+
+      int fieldId = 0;
+      int newOutgoingRowWidth = 0;
+      for (VectorWrapper<?> w : container) {
+        if (w.getValueVector() instanceof FixedWidthVector) {
+          newOutgoingRowWidth += ((FixedWidthVector) w.getValueVector()).getValueWidth();
+          if (fieldId >= numGroupByExprs) {
+            valuesRowWidth += ((FixedWidthVector) w.getValueVector()).getValueWidth();
+          }
+        } else {
+          int columnWidth;
+          if (columnMapping.get(w.getValueVector().getField().getName()) == null) {
+             columnWidth = TypeHelper.getSize(w.getField().getType());
+          } else {
+            RecordBatchSizer.ColumnSize columnSize = getRecordBatchSizer().getColumn(columnMapping.get(w.getValueVector().getField().getName()));
+            if (columnSize == null) {
+              columnWidth = TypeHelper.getSize(w.getField().getType());
+            } else {
+              columnWidth = columnSize.getAllocSizePerEntry();
+            }
+          }
+          newOutgoingRowWidth += columnWidth;
+          if (fieldId >= numGroupByExprs) {
+            valuesRowWidth += columnWidth;
+          }
+        }
+        fieldId++;
+      }
+
+      if (updateIfNeeded(newOutgoingRowWidth)) {
+        // There is an update to outgoing row width.
+        // un comment this if we want to adjust the batch row count of in flight batches.
+        // To keep things simple, we are not doing this adjustment for now.
+        // aggregator.adjustOutputCount(getOutputBatchSize(), getOutgoingRowWidth(), newOutgoingRowWidth);
+      }
+
+      updateIncomingStats();
+      if (logger.isDebugEnabled()) {
+        logger.debug("BATCH_STATS, incoming: {}", getRecordBatchSizer());
+      }
+    }
+  }
+
+  public HashAggBatch(HashAggregate popConfig, RecordBatch incoming, FragmentContext context) {
     super(popConfig, context);
     this.incoming = incoming;
     wasKilled = false;
@@ -104,6 +177,13 @@ public HashAggBatch(HashAggregate popConfig, RecordBatch incoming, FragmentConte
 
     boolean allowed = oContext.getAllocator().setLenient();
     logger.debug("Config: Is allocator lenient? {}", allowed);
+
+    // get the output batch size from config.
+    int configuredBatchSize = (int) context.getOptions().getOption(ExecConstants.OUTPUT_BATCH_SIZE_VALIDATOR);
+    hashAggMemoryManager = new HashAggMemoryManager(configuredBatchSize);
+    logger.debug("BATCH_STATS, configured output batch size: {}", configuredBatchSize);
+
+    columnMapping = CaseInsensitiveMap.newHashMap();
   }
 
   @Override
@@ -137,6 +217,9 @@ public void buildSchema() throws SchemaChangeException {
     for (VectorWrapper<?> w : container) {
       AllocationHelper.allocatePrecomputedChildCount(w.getValueVector(), 0, 0, 0);
     }
+    if (incoming.getRecordCount() > 0) {
+      hashAggMemoryManager.update();
+    }
   }
 
   @Override
@@ -149,14 +232,23 @@ public IterOutcome innerNext() {
     // if aggregation is complete and not all records have been output yet
     if (aggregator.buildComplete() ||
         // or: 1st phase need to return (not fully grouped) partial output due to memory pressure
-        aggregator.earlyOutput()) {
+        aggregator.earlyOutput() ||
+        // or: while handling an EMIT - returning output for that section
+        aggregator.handlingEmit() ) {
       // then output the next batch downstream
       HashAggregator.AggIterOutcome aggOut = aggregator.outputCurrentBatch();
-      // if Batch returned, or end of data - then return the appropriate iter outcome
-      if ( aggOut == HashAggregator.AggIterOutcome.AGG_NONE ) { return IterOutcome.NONE; }
-      if ( aggOut == HashAggregator.AggIterOutcome.AGG_OK ) { return IterOutcome.OK; }
-      // if RESTART - continue below with doWork() - read some spilled partition, just like reading incoming
-      incoming = aggregator.getNewIncoming(); // Restart - incoming was just changed
+      // if Batch returned, or end of data, or Emit - then return the appropriate iter outcome
+      switch ( aggOut ) {
+        case AGG_NONE:
+          return IterOutcome.NONE;
+        case AGG_OK:
+          return IterOutcome.OK;
+        case AGG_EMIT:
+          return IterOutcome.EMIT;
+        default: // i.e. RESTART
+          // if RESTART - continue below with doWork() - read some spilled partition, just like reading incoming
+          incoming = aggregator.getNewIncoming(); // Restart - incoming was just changed
+      }
     }
 
     if (wasKilled) { // if kill() was called before, then finish up
@@ -227,10 +319,12 @@ private HashAggregator createAggregatorInternal() throws SchemaChangeException,
     ClassGenerator<HashAggregator> cg = top.getRoot();
     ClassGenerator<HashAggregator> cgInner = cg.getInnerGenerator("BatchHolder");
     top.plainJavaCapable(true);
+    // Uncomment the following line to allow debugging of the template code
+    // top.saveCodeForDebugging(true);
     container.clear();
 
-    int numGroupByExprs = (popConfig.getGroupByExprs() != null) ? popConfig.getGroupByExprs().size() : 0;
-    int numAggrExprs = (popConfig.getAggrExprs() != null) ? popConfig.getAggrExprs().size() : 0;
+    numGroupByExprs = (popConfig.getGroupByExprs() != null) ? popConfig.getGroupByExprs().size() : 0;
+    numAggrExprs = (popConfig.getAggrExprs() != null) ? popConfig.getAggrExprs().size() : 0;
     aggrExprs = new LogicalExpression[numAggrExprs];
     groupByOutFieldIds = new TypedFieldId[numGroupByExprs];
     aggrOutFieldIds = new TypedFieldId[numAggrExprs];
@@ -253,13 +347,13 @@ private HashAggregator createAggregatorInternal() throws SchemaChangeException,
 
       // add this group-by vector to the output container
       groupByOutFieldIds[i] = container.add(vv);
+      columnMapping.put(outputField.getName(), ne.getExpr().toString().replace('`',' ').trim());
     }
 
     int extraNonNullColumns = 0; // each of SUM, MAX and MIN gets an extra bigint column
     for (i = 0; i < numAggrExprs; i++) {
       NamedExpression ne = popConfig.getAggrExprs().get(i);
-      final LogicalExpression expr =
-          ExpressionTreeMaterializer.materialize(ne.getExpr(), incoming, collector, context.getFunctionRegistry());
+      final LogicalExpression expr = ExpressionTreeMaterializer.materialize(ne.getExpr(), incoming, collector, context.getFunctionRegistry());
 
       if (expr instanceof IfExpression) {
         throw UserException.unsupportedError(new UnsupportedOperationException("Union type not supported in aggregate functions")).build(logger);
@@ -273,16 +367,28 @@ private HashAggregator createAggregatorInternal() throws SchemaChangeException,
         continue;
       }
 
-      if ( expr instanceof FunctionHolderExpression ) {
-         String funcName = ((FunctionHolderExpression) expr).getName();
-         if ( funcName.equals("sum") || funcName.equals("max") || funcName.equals("min") ) {extraNonNullColumns++;}
-      }
       final MaterializedField outputField = MaterializedField.create(ne.getRef().getAsNamePart().getName(), expr.getMajorType());
-      @SuppressWarnings("resource")
-      ValueVector vv = TypeHelper.getNewVector(outputField, oContext.getAllocator());
+      @SuppressWarnings("resource") ValueVector vv = TypeHelper.getNewVector(outputField, oContext.getAllocator());
       aggrOutFieldIds[i] = container.add(vv);
 
       aggrExprs[i] = new ValueVectorWriteExpression(aggrOutFieldIds[i], expr, true);
+
+      if (expr instanceof FunctionHolderExpression) {
+        String funcName = ((FunctionHolderExpression) expr).getName();
+        if (funcName.equals("sum") || funcName.equals("max") || funcName.equals("min")) {
+          extraNonNullColumns++;
+        }
+        if (((FunctionCall) ne.getExpr()).args.get(0) instanceof SchemaPath) {
+          columnMapping.put(outputField.getName(), ((SchemaPath) ((FunctionCall) ne.getExpr()).args.get(0)).getAsNamePart().getName());
+        }  else if (((FunctionCall) ne.getExpr()).args.get(0) instanceof FunctionCall) {
+          FunctionCall functionCall = (FunctionCall) ((FunctionCall) ne.getExpr()).args.get(0);
+          if (functionCall.args.get(0) instanceof SchemaPath) {
+            columnMapping.put(outputField.getName(), ((SchemaPath) functionCall.args.get(0)).getAsNamePart().getName());
+          }
+        }
+      } else {
+        columnMapping.put(outputField.getName(), ne.getRef().getAsNamePart().getName());
+      }
     }
 
     setupUpdateAggrValues(cgInner);
@@ -335,11 +441,32 @@ private void setupGetIndex(ClassGenerator<HashAggregator> cg) {
     }
   }
 
+  private void updateStats() {
+    stats.setLongStat(HashAggTemplate.Metric.INPUT_BATCH_COUNT, hashAggMemoryManager.getNumIncomingBatches());
+    stats.setLongStat(HashAggTemplate.Metric.AVG_INPUT_BATCH_BYTES, hashAggMemoryManager.getAvgInputBatchSize());
+    stats.setLongStat(HashAggTemplate.Metric.AVG_INPUT_ROW_BYTES, hashAggMemoryManager.getAvgInputRowWidth());
+    stats.setLongStat(HashAggTemplate.Metric.INPUT_RECORD_COUNT, hashAggMemoryManager.getTotalInputRecords());
+    stats.setLongStat(HashAggTemplate.Metric.OUTPUT_BATCH_COUNT, hashAggMemoryManager.getNumOutgoingBatches());
+    stats.setLongStat(HashAggTemplate.Metric.AVG_OUTPUT_BATCH_BYTES, hashAggMemoryManager.getAvgOutputBatchSize());
+    stats.setLongStat(HashAggTemplate.Metric.AVG_OUTPUT_ROW_BYTES, hashAggMemoryManager.getAvgOutputRowWidth());
+    stats.setLongStat(HashAggTemplate.Metric.OUTPUT_RECORD_COUNT, hashAggMemoryManager.getTotalOutputRecords());
+
+    if (logger.isDebugEnabled()) {
+      logger.debug("BATCH_STATS, incoming aggregate: count : {}, avg bytes : {},  avg row bytes : {}, record count : {}",
+        hashAggMemoryManager.getNumIncomingBatches(), hashAggMemoryManager.getAvgInputBatchSize(),
+        hashAggMemoryManager.getAvgInputRowWidth(), hashAggMemoryManager.getTotalInputRecords());
+
+      logger.debug("BATCH_STATS, outgoing aggregate: count : {}, avg bytes : {},  avg row bytes : {}, record count : {}",
+        hashAggMemoryManager.getNumOutgoingBatches(), hashAggMemoryManager.getAvgOutputBatchSize(),
+        hashAggMemoryManager.getAvgOutputRowWidth(), hashAggMemoryManager.getTotalOutputRecords());
+    }
+  }
   @Override
   public void close() {
     if (aggregator != null) {
       aggregator.cleanup();
     }
+    updateStats();
     super.close();
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java
index 258e8d0856..2f3bc23da3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java
@@ -79,6 +79,7 @@
 
 import org.apache.drill.exec.vector.VariableWidthVector;
 
+import static org.apache.drill.exec.physical.impl.common.HashTable.BATCH_MASK;
 import static org.apache.drill.exec.record.RecordBatch.MAX_BATCH_SIZE;
 
 public abstract class HashAggTemplate implements HashAggregator {
@@ -159,8 +160,6 @@
   private int operatorId; // for the spill file name
 
   private IndexPointer htIdxHolder; // holder for the Hashtable's internal index returned by put()
-  private IndexPointer outStartIdxHolder;
-  private IndexPointer outNumRecordsHolder;
   private int numGroupByOutFields = 0; // Note: this should be <= number of group-by fields
   private TypedFieldId[] groupByOutFieldIds;
 
@@ -168,6 +167,7 @@
   private boolean allFlushed = false;
   private boolean buildComplete = false;
   private boolean handlingSpills = false; // True once starting to process spill files
+  private boolean handleEmit = false; // true after receiving an EMIT, till finish handling it
 
   private OperatorStats stats = null;
   private HashTableStats htStats = new HashTableStats();
@@ -184,7 +184,15 @@
                       // then later re-read. So, disk I/O is twice this amount.
                       // For first phase aggr -- this is an estimate of the amount of data
                       // returned early (analogous to a spill in the 2nd phase).
-    SPILL_CYCLE       // 0 - no spill, 1 - spill, 2 - SECONDARY, 3 - TERTIARY
+    SPILL_CYCLE,       // 0 - no spill, 1 - spill, 2 - SECONDARY, 3 - TERTIARY
+    INPUT_BATCH_COUNT,
+    AVG_INPUT_BATCH_BYTES,
+    AVG_INPUT_ROW_BYTES,
+    INPUT_RECORD_COUNT,
+    OUTPUT_BATCH_COUNT,
+    AVG_OUTPUT_BATCH_BYTES,
+    AVG_OUTPUT_ROW_BYTES,
+    OUTPUT_RECORD_COUNT;
     ;
 
     @Override
@@ -194,16 +202,29 @@ public int metricId() {
   }
 
   public class BatchHolder {
-
     private VectorContainer aggrValuesContainer; // container for aggr values (workspace variables)
     private int maxOccupiedIdx = -1;
-    private int batchOutputCount = 0;
+    private int targetBatchRowCount = 0;
+
+    public int getTargetBatchRowCount() {
+      return targetBatchRowCount;
+    }
+
+    public void setTargetBatchRowCount(int batchRowCount) {
+      this.targetBatchRowCount = batchRowCount;
+    }
+
+    public int getCurrentRowCount() {
+      return (maxOccupiedIdx + 1);
+    }
 
     @SuppressWarnings("resource")
-    public BatchHolder() {
+    public BatchHolder(int batchRowCount) {
 
       aggrValuesContainer = new VectorContainer();
       boolean success = false;
+      this.targetBatchRowCount = batchRowCount;
+
       try {
         ValueVector vector;
 
@@ -219,12 +240,12 @@ public BatchHolder() {
           // BatchHolder in HashTable, causing the HashTable to be space inefficient. So it is better to allocate space
           // to fit as close to as BATCH_SIZE records.
           if (vector instanceof FixedWidthVector) {
-            ((FixedWidthVector) vector).allocateNew(HashTable.BATCH_SIZE);
+            ((FixedWidthVector) vector).allocateNew(batchRowCount);
           } else if (vector instanceof VariableWidthVector) {
             // This case is never used .... a varchar falls under ObjectVector which is allocated on the heap !
-            ((VariableWidthVector) vector).allocateNew(maxColumnWidth, HashTable.BATCH_SIZE);
+            ((VariableWidthVector) vector).allocateNew(maxColumnWidth, batchRowCount);
           } else if (vector instanceof ObjectVector) {
-            ((ObjectVector) vector).allocateNew(HashTable.BATCH_SIZE);
+            ((ObjectVector) vector).allocateNew(batchRowCount);
           } else {
             vector.allocateNew();
           }
@@ -251,17 +272,12 @@ private void setup() {
       catch (SchemaChangeException sc) { throw new UnsupportedOperationException(sc);}
     }
 
-    private void outputValues(IndexPointer outStartIdxHolder, IndexPointer outNumRecordsHolder) {
-      outStartIdxHolder.value = batchOutputCount;
-      outNumRecordsHolder.value = 0;
-      for (int i = batchOutputCount; i <= maxOccupiedIdx; i++) {
-        try { outputRecordValues(i, batchOutputCount); }
-        catch (SchemaChangeException sc) { throw new UnsupportedOperationException(sc);}
-        if (EXTRA_DEBUG_2) {
-          logger.debug("Outputting values to output index: {}", batchOutputCount);
+    private void outputValues() {
+      for (int i = 0; i <= maxOccupiedIdx; i++) {
+        try {
+          outputRecordValues(i, i);
         }
-        batchOutputCount++;
-        outNumRecordsHolder.value++;
+        catch (SchemaChangeException sc) { throw new UnsupportedOperationException(sc);}
       }
     }
 
@@ -274,7 +290,7 @@ private int getNumGroups() {
     }
 
     private int getNumPendingOutput() {
-      return getNumGroups() - batchOutputCount;
+      return getNumGroups();
     }
 
     // Code-generated methods (implemented in HashAggBatch)
@@ -348,9 +364,6 @@ public void setup(HashAggregate hashAggrConfig, HashTableConfig htConfig, Fragme
     }
 
     this.htIdxHolder = new IndexPointer();
-    this.outStartIdxHolder = new IndexPointer();
-    this.outNumRecordsHolder = new IndexPointer();
-
     materializedValueFields = new MaterializedField[valueFieldIds.size()];
 
     if (valueFieldIds.size() > 0) {
@@ -512,14 +525,14 @@ private void initializeSetup(RecordBatch newIncoming) throws SchemaChangeExcepti
   private void updateEstMaxBatchSize(RecordBatch incoming) {
     if ( estMaxBatchSize > 0 ) { return; }  // no handling of a schema (or varchar) change
     // Use the sizer to get the input row width and the length of the longest varchar column
-    RecordBatchSizer sizer = new RecordBatchSizer(incoming);
+    RecordBatchSizer sizer = outgoing.getRecordBatchMemoryManager().getRecordBatchSizer();
     logger.trace("Incoming sizer: {}",sizer);
     // An empty batch only has the schema, can not tell actual length of varchars
     // else use the actual varchars length, each capped at 50 (to match the space allocation)
-    long estInputRowWidth = sizer.rowCount() == 0 ? sizer.stdRowWidth() : sizer.netRowWidthCap50();
+    long estInputRowWidth = sizer.rowCount() == 0 ? sizer.getStdRowWidth() : sizer.getNetRowWidthCap50();
 
     // Get approx max (varchar) column width to get better memory allocation
-    maxColumnWidth = Math.max(sizer.maxAvgColumnSize(), VARIABLE_MIN_WIDTH_VALUE_SIZE);
+    maxColumnWidth = Math.max(sizer.getMaxAvgColumnSize(), VARIABLE_MIN_WIDTH_VALUE_SIZE);
     maxColumnWidth = Math.min(maxColumnWidth, VARIABLE_MAX_WIDTH_VALUE_SIZE);
 
     //
@@ -604,32 +617,36 @@ public AggOutcome doWork() {
       for (VectorWrapper<?> v : incoming) {
         v.getValueVector().clear();
       }
-      //
-      // Get the NEXT input batch, initially from the upstream, later (if there was a spill)
-      // from one of the spill files (The spill case is handled differently here to avoid
-      // collecting stats on the spilled records)
-      //
-      long memAllocBeforeNext = allocator.getAllocatedMemory();
-      if ( handlingSpills ) {
-        outcome = incoming.next(); // get it from the SpilledRecordBatch
-      } else {
-        // Get the next RecordBatch from the incoming (i.e. upstream operator)
-        outcome = outgoing.next(0, incoming);
-      }
-      long memAllocAfterNext = allocator.getAllocatedMemory();
-      long incomingBatchSize = memAllocAfterNext - memAllocBeforeNext;
-
-      // If incoming batch is bigger than our estimate - adjust the estimate to match
-      if ( estMaxBatchSize < incomingBatchSize) {
-        logger.debug("Found a bigger next {} batch: {} , prior estimate was: {}, mem allocated {}",handlingSpills ? "spill" : "incoming",
-            incomingBatchSize, estMaxBatchSize, memAllocAfterNext);
-        estMaxBatchSize = incomingBatchSize;
-      }
 
-      if (EXTRA_DEBUG_1) {
-        logger.debug("Received IterOutcome of {}", outcome);
+      if ( handleEmit ) {
+        outcome = IterOutcome.NONE; // finished behaving like OK, now behave like NONE
       }
+      else {
+        //
+        // Get the NEXT input batch, initially from the upstream, later (if there was a spill)
+        // from one of the spill files (The spill case is handled differently here to avoid
+        // collecting stats on the spilled records)
+        //
+        long memAllocBeforeNext = allocator.getAllocatedMemory();
+        if (handlingSpills) {
+          outcome = incoming.next(); // get it from the SpilledRecordBatch
+        } else {
+          // Get the next RecordBatch from the incoming (i.e. upstream operator)
+          outcome = outgoing.next(0, incoming);
+        }
+        long memAllocAfterNext = allocator.getAllocatedMemory();
+        long incomingBatchSize = memAllocAfterNext - memAllocBeforeNext;
 
+        // If incoming batch is bigger than our estimate - adjust the estimate to match
+        if (estMaxBatchSize < incomingBatchSize) {
+          logger.debug("Found a bigger next {} batch: {} , prior estimate was: {}, mem allocated {}", handlingSpills ? "spill" : "incoming", incomingBatchSize, estMaxBatchSize, memAllocAfterNext);
+          estMaxBatchSize = incomingBatchSize;
+        }
+
+        if (EXTRA_DEBUG_1) {
+          logger.debug("Received IterOutcome of {}", outcome);
+        }
+      }
       // Handle various results from getting the next batch
       switch (outcome) {
         case OUT_OF_MEMORY:
@@ -644,7 +661,13 @@ public AggOutcome doWork() {
           // TODO: new schema case needs to be handled appropriately
           return AggOutcome.UPDATE_AGGREGATOR;
 
+        case EMIT:
+          handleEmit = true;
+          // remember EMIT, but continue like handling OK
+
         case OK:
+          outgoing.getRecordBatchMemoryManager().update();
+
           currentBatchRecordCount = incoming.getRecordCount(); // size of next batch
 
           resetIndex(); // initialize index (a new batch needs to be processed)
@@ -657,21 +680,36 @@ public AggOutcome doWork() {
         case NONE:
           resetIndex(); // initialize index (in case spill files need to be processed)
 
-          buildComplete = true;
+          // Either flag buildComplete or handleEmit (or earlyOutput) would cause returning of
+          // the outgoing batch downstream (see innerNext() in HashAggBatch).
+          buildComplete = true ; // now should go and return outgoing
 
+          if ( handleEmit ) {
+            buildComplete = false; // This was not a real NONE - more incoming is expected
+            // don't aggregate this incoming batch again (in the loop above; when doWork() is called again)
+            currentBatchRecordCount = 0;
+          }
           updateStats(htables);
 
           // output the first batch; remaining batches will be output
           // in response to each next() call by a downstream operator
           AggIterOutcome aggOutcome = outputCurrentBatch();
 
-          if ( aggOutcome == AggIterOutcome.AGG_RESTART ) {
-            // Output of first batch returned a RESTART (all new partitions were spilled)
-            return AggOutcome.CALL_WORK_AGAIN; // need to read/process the next partition
+          switch ( aggOutcome ) {
+            case AGG_RESTART:
+              // Output of first batch returned a RESTART (all new partitions were spilled)
+              return AggOutcome.CALL_WORK_AGAIN; // need to read/process the next partition
+            case AGG_EMIT:
+              // Following an incoming EMIT, if the output was only a single batch
+              // outcome is set to IterOutcome.EMIT;
+              break;
+            case AGG_NONE: // no output
+              break;
+            default:
+              // Regular output (including after EMIT, when more output batches are planned)
+              outcome = IterOutcome.OK;
           }
 
-          if ( aggOutcome != AggIterOutcome.AGG_NONE ) { outcome = IterOutcome.OK; }
-
           return AggOutcome.RETURN_OUTCOME;
 
         case STOP:
@@ -750,6 +788,7 @@ private void allocateOutgoing(int records) {
       logger.trace("Output values allocated {} but the estimate was only {}. Adjusting ...",memAdded,estOutgoingAllocSize);
       estOutgoingAllocSize = memAdded;
     }
+    outContainer.setRecordCount(records);
     // try to restore the reserve
     restoreReservedMemory();
   }
@@ -764,6 +803,22 @@ public int getOutputCount() {
     return lastBatchOutputCount;
   }
 
+  @Override
+  public void adjustOutputCount(int outputBatchSize, int oldRowWidth, int newRowWidth) {
+    for (int i = 0; i < numPartitions; i++ ) {
+      if (batchHolders[i] == null || batchHolders[i].size() == 0) {
+        continue;
+      }
+      BatchHolder bh = batchHolders[i].get(batchHolders[i].size()-1);
+      // Divide remaining memory by new row width.
+      final int remainingRows = RecordBatchSizer.safeDivide(Math.max((outputBatchSize - (bh.getCurrentRowCount() * oldRowWidth)), 0), newRowWidth);
+      // Do not go beyond the current target row count as this might cause reallocs for fixed width vectors.
+      final int newRowCount = Math.min(bh.getTargetBatchRowCount(), bh.getCurrentRowCount() + remainingRows);
+      bh.setTargetBatchRowCount(newRowCount);
+      htables[i].setTargetBatchRowCount(newRowCount);
+    }
+  }
+
   @Override
   public void cleanup() {
     if ( schema == null ) { return; } // not set up; nothing to clean
@@ -811,8 +866,6 @@ public void cleanup() {
     spillSet.close(); // delete the spill directory(ies)
     htIdxHolder = null;
     materializedValueFields = null;
-    outStartIdxHolder = null;
-    outNumRecordsHolder = null;
   }
 
   // First free the memory used by the given (spilled) partition (i.e., hash table plus batches)
@@ -828,6 +881,7 @@ private void reinitPartition(int part) /* throws SchemaChangeException /*, IOExc
     }
     batchHolders[part] = new ArrayList<BatchHolder>(); // First BatchHolder is created when the first put request is received.
 
+    outBatchIndex[part] = 0;
     // in case the reserve memory was used, try to restore
     restoreReservedMemory();
   }
@@ -937,17 +991,14 @@ private void spillAPartition(int part) {
     for (int currOutBatchIndex = 0; currOutBatchIndex < currPartition.size(); currOutBatchIndex++ ) {
 
       // get the number of records in the batch holder that are pending output
-      int numPendingOutput = currPartition.get(currOutBatchIndex).getNumPendingOutput();
+      int numOutputRecords = currPartition.get(currOutBatchIndex).getNumPendingOutput();
 
-      rowsInPartition += numPendingOutput;  // for logging
-      rowsSpilled += numPendingOutput;
+      rowsInPartition += numOutputRecords;  // for logging
+      rowsSpilled += numOutputRecords;
 
-      allocateOutgoing(numPendingOutput);
-
-      currPartition.get(currOutBatchIndex).outputValues(outStartIdxHolder, outNumRecordsHolder);
-      int numOutputRecords = outNumRecordsHolder.value;
-
-      this.htables[part].outputKeys(currOutBatchIndex, this.outContainer, outStartIdxHolder.value, outNumRecordsHolder.value, numPendingOutput);
+      allocateOutgoing(numOutputRecords);
+      currPartition.get(currOutBatchIndex).outputValues();
+      this.htables[part].outputKeys(currOutBatchIndex, this.outContainer, numOutputRecords);
 
       // set the value count for outgoing batch value vectors
       /* int i = 0; */
@@ -967,8 +1018,8 @@ private void spillAPartition(int part) {
         */
       }
 
-      outContainer.setRecordCount(numPendingOutput);
-      WritableBatch batch = WritableBatch.getBatchNoHVWrap(numPendingOutput, outContainer, false);
+      outContainer.setRecordCount(numOutputRecords);
+      WritableBatch batch = WritableBatch.getBatchNoHVWrap(numOutputRecords, outContainer, false);
       try {
         writers[part].write(batch, null);
       } catch (IOException ioe) {
@@ -979,7 +1030,7 @@ private void spillAPartition(int part) {
         batch.clear();
       }
       outContainer.zeroVectors();
-      logger.trace("HASH AGG: Took {} us to spill {} records", writers[part].time(TimeUnit.MICROSECONDS), numPendingOutput);
+      logger.trace("HASH AGG: Took {} us to spill {} records", writers[part].time(TimeUnit.MICROSECONDS), numOutputRecords);
     }
 
     spilledBatchesCount[part] += currPartition.size(); // update count of spilled batches
@@ -987,9 +1038,9 @@ private void spillAPartition(int part) {
     logger.trace("HASH AGG: Spilled {} rows from {} batches of partition {}", rowsInPartition, currPartition.size(), part);
   }
 
-  private void addBatchHolder(int part) {
+  private void addBatchHolder(int part, int batchRowCount) {
 
-    BatchHolder bh = newBatchHolder();
+    BatchHolder bh = newBatchHolder(batchRowCount);
     batchHolders[part].add(bh);
     if (EXTRA_DEBUG_1) {
       logger.debug("HashAggregate: Added new batch; num batches = {}.", batchHolders[part].size());
@@ -999,8 +1050,8 @@ private void addBatchHolder(int part) {
   }
 
   // These methods are overridden in the generated class when created as plain Java code.
-  protected BatchHolder newBatchHolder() {
-    return new BatchHolder();
+  protected BatchHolder newBatchHolder(int batchRowCount) {
+    return new BatchHolder(batchRowCount);
   }
 
   /**
@@ -1011,6 +1062,23 @@ protected BatchHolder newBatchHolder() {
   @Override
   public AggIterOutcome outputCurrentBatch() {
 
+    // Handle the case of an EMIT with an empty batch
+    if ( handleEmit && ( batchHolders == null || batchHolders[0].size() == 0 ) ) {
+      lastBatchOutputCount = 0; // empty
+      allocateOutgoing(0);
+      for (VectorWrapper<?> v : outgoing) {
+        v.getValueVector().getMutator().setValueCount(0);
+      }
+      outgoing.getContainer().setRecordCount(0);
+      // When returning the last outgoing batch (following an incoming EMIT), then replace OK with EMIT
+      this.outcome = IterOutcome.EMIT;
+      handleEmit = false; // finish handling EMIT
+      if ( outBatchIndex != null ) {
+        outBatchIndex[0] = 0; // reset, for the next EMIT
+      }
+      return AggIterOutcome.AGG_EMIT;
+    }
+
     // when incoming was an empty batch, just finish up
     if ( schema == null ) {
       logger.trace("Incoming was empty; output is an empty batch.");
@@ -1119,20 +1187,21 @@ public AggIterOutcome outputCurrentBatch() {
 
     allocateOutgoing(numPendingOutput);
 
-    currPartition.get(currOutBatchIndex).outputValues(outStartIdxHolder, outNumRecordsHolder);
-    int numOutputRecords = outNumRecordsHolder.value;
-
-    if (EXTRA_DEBUG_1) {
-      logger.debug("After output values: outStartIdx = {}, outNumRecords = {}", outStartIdxHolder.value, outNumRecordsHolder.value);
-    }
-
-    this.htables[partitionToReturn].outputKeys(currOutBatchIndex, this.outContainer, outStartIdxHolder.value, outNumRecordsHolder.value, numPendingOutput);
+    currPartition.get(currOutBatchIndex).outputValues();
+    int numOutputRecords = numPendingOutput;
+    this.htables[partitionToReturn].outputKeys(currOutBatchIndex, this.outContainer, numPendingOutput);
 
     // set the value count for outgoing batch value vectors
     for (VectorWrapper<?> v : outgoing) {
       v.getValueVector().getMutator().setValueCount(numOutputRecords);
     }
 
+    outgoing.getRecordBatchMemoryManager().updateOutgoingStats(numOutputRecords);
+
+    if (logger.isDebugEnabled()) {
+      logger.debug("BATCH_STATS, outgoing: {}", new RecordBatchSizer(outgoing));
+    }
+
     this.outcome = IterOutcome.OK;
 
     if ( EXTRA_DEBUG_SPILL && is2ndPhase ) {
@@ -1164,6 +1233,13 @@ public AggIterOutcome outputCurrentBatch() {
         outBatchIndex[earlyPartition] = 0; // reset, for next time
         earlyOutput = false ; // done with early output
       }
+      else if ( handleEmit ) {
+        // When returning the last outgoing batch (following an incoming EMIT), then replace OK with EMIT
+        this.outcome = IterOutcome.EMIT;
+        handleEmit = false; // finished handling EMIT
+        outBatchIndex[partitionToReturn] = 0; // reset, for the next EMIT
+        return AggIterOutcome.AGG_EMIT;
+      }
       else if ( (partitionToReturn + 1 == numPartitions) && spilledPartitionsList.isEmpty() ) { // last partition ?
 
         allFlushed = true; // next next() call will return NONE
@@ -1187,6 +1263,12 @@ public boolean allFlushed() {
   public boolean buildComplete() {
     return buildComplete;
   }
+
+  @Override
+  public boolean handlingEmit() {
+    return handleEmit;
+  }
+
   @Override
   public boolean earlyOutput() { return earlyOutput; }
 
@@ -1216,6 +1298,10 @@ private String getOOMErrorMsg(String prefix) {
     return errmsg;
   }
 
+  private int getTargetBatchCount() {
+    return outgoing.getOutputRowCount();
+  }
+
   // Check if a group is present in the hash table; if not, insert it in the hash table.
   // The htIdxHolder contains the index of the group in the hash table container; this same
   // index is also used for the aggregation values maintained by the hash aggregate.
@@ -1283,7 +1369,7 @@ private void checkGroupAndAggrValues(int incomingRowIdx) {
     // ==========================================
     try {
 
-      putStatus = htables[currentPartition].put(incomingRowIdx, htIdxHolder, hashCode);
+      putStatus = htables[currentPartition].put(incomingRowIdx, htIdxHolder, hashCode, getTargetBatchCount());
 
     } catch (RetryAfterSpillException re) {
       if ( ! canSpill ) { throw new OutOfMemoryException(getOOMErrorMsg("Can not spill")); }
@@ -1317,7 +1403,7 @@ private void checkGroupAndAggrValues(int incomingRowIdx) {
 
         useReservedValuesMemory(); // try to preempt an OOM by using the reserve
 
-        addBatchHolder(currentPartition);  // allocate a new (internal) values batch
+        addBatchHolder(currentPartition, getTargetBatchCount());  // allocate a new (internal) values batch
 
         restoreReservedMemory(); // restore the reserve, if possible
         // A reason to check for a spill - In case restore-reserve failed
@@ -1353,8 +1439,8 @@ private void checkGroupAndAggrValues(int incomingRowIdx) {
     // Locate the matching aggregate columns and perform the aggregation
     // =================================================================
     int currentIdx = htIdxHolder.value;
-    BatchHolder bh = batchHolders[currentPartition].get((currentIdx >>> 16) & HashTable.BATCH_MASK);
-    int idxWithinBatch = currentIdx & HashTable.BATCH_MASK;
+    BatchHolder bh = batchHolders[currentPartition].get((currentIdx >>> 16) & BATCH_MASK);
+    int idxWithinBatch = currentIdx & BATCH_MASK;
     if (bh.updateAggrValues(incomingRowIdx, idxWithinBatch)) {
       numGroupedRecords++;
     }
@@ -1464,7 +1550,8 @@ private void spillIfNeeded(int currentPartition, boolean forceSpill) {
    * @param htables
    */
   private void updateStats(HashTable[] htables) {
-    if ( cycleNum > 0 ) { return; } // These stats are only for before processing spilled files
+    if ( cycleNum > 0 ||  // These stats are only for before processing spilled files
+      handleEmit ) { return; } // and no stats collecting when handling an EMIT
     long numSpilled = 0;
     HashTableStats newStats = new HashTableStats();
     // sum the stats from all the partitions
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggregator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggregator.java
index e5167d8544..f58be89291 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggregator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggregator.java
@@ -43,8 +43,8 @@
   }
 
   // For returning results from outputCurrentBatch
-  // OK - batch returned, NONE - end of data, RESTART - call again
-  enum AggIterOutcome { AGG_OK, AGG_NONE, AGG_RESTART }
+  // OK - batch returned, NONE - end of data, RESTART - call again, EMIT - like OK but EMIT
+  enum AggIterOutcome { AGG_OK, AGG_NONE, AGG_RESTART , AGG_EMIT }
 
   void setup(HashAggregate hashAggrConfig, HashTableConfig htConfig, FragmentContext context, OperatorContext oContext, RecordBatch incoming, HashAggBatch outgoing,
              LogicalExpression[] valueExprs, List<TypedFieldId> valueFieldIds, TypedFieldId[] keyFieldIds, VectorContainer outContainer, int extraRowBytes) throws SchemaChangeException, IOException, ClassTransformationException;
@@ -61,9 +61,13 @@ void setup(HashAggregate hashAggrConfig, HashTableConfig htConfig, FragmentConte
 
   boolean buildComplete();
 
+  boolean handlingEmit();
+
   AggIterOutcome outputCurrentBatch();
 
   boolean earlyOutput();
 
   RecordBatch getNewIncoming();
+
+  void adjustOutputCount(int outputBatchSize, int oldRowWidth, int newRowWidth);
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java
index 34ab97e2c0..caeed50df3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java
@@ -18,7 +18,9 @@
 package org.apache.drill.exec.physical.impl.aggregate;
 
 import java.io.IOException;
+import java.util.List;
 
+import com.google.common.collect.Lists;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.expression.ErrorCollector;
@@ -35,6 +37,7 @@
 import org.apache.drill.exec.expr.ClassGenerator;
 import org.apache.drill.exec.expr.ClassGenerator.HoldingContainer;
 import org.apache.drill.exec.expr.CodeGenerator;
+import org.apache.drill.exec.expr.DrillFuncHolderExpr;
 import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
 import org.apache.drill.exec.expr.HoldingContainerExpression;
 import org.apache.drill.exec.expr.TypeHelper;
@@ -50,21 +53,26 @@
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.RecordBatch;
 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.selection.SelectionVector2;
 import org.apache.drill.exec.record.selection.SelectionVector4;
 import org.apache.drill.exec.vector.AllocationHelper;
 import org.apache.drill.exec.vector.FixedWidthVector;
+import org.apache.drill.exec.vector.UntypedNullHolder;
+import org.apache.drill.exec.vector.UntypedNullVector;
 import org.apache.drill.exec.vector.ValueVector;
 
 import com.sun.codemodel.JExpr;
 import com.sun.codemodel.JVar;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter;
 
 public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StreamingAggBatch.class);
 
   private StreamingAggregator aggregator;
   private final RecordBatch incoming;
+  private List<BaseWriter.ComplexWriter> complexWriters;
   private boolean done = false;
   private boolean first = true;
   private int recordCount = 0;
@@ -106,6 +114,11 @@ public int getRecordCount() {
     return recordCount;
   }
 
+  @Override
+  public VectorContainer getOutgoingContainer() {
+    return this.container;
+  }
+
   @Override
   public void buildSchema() throws SchemaChangeException {
     IterOutcome outcome = next(incoming);
@@ -131,6 +144,10 @@ public void buildSchema() throws SchemaChangeException {
     for (final VectorWrapper<?> w : container) {
       w.getValueVector().allocateNew();
     }
+
+    if (complexWriters != null) {
+      container.buildSchema(SelectionVectorMode.NONE);
+    }
   }
 
   @Override
@@ -177,7 +194,6 @@ public IterOutcome innerNext() {
         throw new IllegalStateException(String.format("unknown outcome %s", outcome));
       }
     }
-
     AggOutcome out = aggregator.doWork();
     recordCount = aggregator.getOutputCount();
     logger.debug("Aggregator response {}, records {}", out, aggregator.getOutputCount());
@@ -191,6 +207,11 @@ public IterOutcome innerNext() {
       // fall through
     case RETURN_OUTCOME:
       IterOutcome outcome = aggregator.getOutcome();
+      // In case of complex writer expression, vectors would be added to batch run-time.
+      // We have to re-build the schema.
+      if (complexWriters != null) {
+        container.buildSchema(SelectionVectorMode.NONE);
+      }
       if (outcome == IterOutcome.NONE && first) {
         first = false;
         done = true;
@@ -213,6 +234,14 @@ public IterOutcome innerNext() {
     }
   }
 
+  private void allocateComplexWriters() {
+    // Allocate the complex writers before processing the incoming batch
+    if (complexWriters != null) {
+      for (final BaseWriter.ComplexWriter writer : complexWriters) {
+        writer.allocate();
+      }
+    }
+  }
 
   /**
    * Method is invoked when we have a straight aggregate (no group by expression) and our input is empty.
@@ -272,9 +301,15 @@ private boolean createAggregator() {
     }
   }
 
+  public void addComplexWriter(final BaseWriter.ComplexWriter writer) {
+    complexWriters.add(writer);
+  }
+
   private StreamingAggregator createAggregatorInternal() throws SchemaChangeException, ClassTransformationException, IOException{
     ClassGenerator<StreamingAggregator> cg = CodeGenerator.getRoot(StreamingAggTemplate.TEMPLATE_DEFINITION, context.getOptions());
     cg.getCodeGenerator().plainJavaCapable(true);
+    // Uncomment out this line to debug the generated code.
+    //cg.getCodeGenerator().saveCodeForDebugging(true);
     container.clear();
 
     LogicalExpression[] keyExprs = new LogicalExpression[popConfig.getKeys().size()];
@@ -307,12 +342,29 @@ private StreamingAggregator createAggregatorInternal() throws SchemaChangeExcept
         continue;
       }
 
-      final MaterializedField outputField = MaterializedField.create(ne.getRef().getLastSegment().getNameSegment().getPath(),
-                                                                      expr.getMajorType());
-      @SuppressWarnings("resource")
-      ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
-      TypedFieldId id = container.add(vector);
-      valueExprs[i] = new ValueVectorWriteExpression(id, expr, true);
+      /* Populate the complex writers for complex exprs */
+      if (expr instanceof DrillFuncHolderExpr &&
+          ((DrillFuncHolderExpr) expr).getHolder().isComplexWriterFuncHolder()) {
+        // Need to process ComplexWriter function evaluation.
+        // Lazy initialization of the list of complex writers, if not done yet.
+        if (complexWriters == null) {
+          complexWriters = Lists.newArrayList();
+        } else {
+          complexWriters.clear();
+        }
+        // The reference name will be passed to ComplexWriter, used as the name of the output vector from the writer.
+        ((DrillFuncHolderExpr) expr).getFieldReference(ne.getRef());
+        MaterializedField field = MaterializedField.create(ne.getRef().getAsNamePart().getName(), UntypedNullHolder.TYPE);
+        container.add(new UntypedNullVector(field, container.getAllocator()));
+        valueExprs[i] = expr;
+      } else {
+        final MaterializedField outputField = MaterializedField.create(ne.getRef().getLastSegment().getNameSegment().getPath(),
+            expr.getMajorType());
+        @SuppressWarnings("resource")
+        ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
+        TypedFieldId id = container.add(vector);
+        valueExprs[i] = new ValueVectorWriteExpression(id, expr, true);
+      }
     }
 
     if (collector.hasErrors()) {
@@ -331,6 +383,7 @@ private StreamingAggregator createAggregatorInternal() throws SchemaChangeExcept
     container.buildSchema(SelectionVectorMode.NONE);
     StreamingAggregator agg = context.getImplementationClass(cg);
     agg.setup(oContext, incoming, this);
+    allocateComplexWriters();
     return agg;
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashPartition.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashPartition.java
index e525530b31..eaccd33552 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashPartition.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashPartition.java
@@ -53,6 +53,8 @@
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 
+import static org.apache.drill.exec.physical.impl.common.HashTable.BATCH_SIZE;
+
 /**
  * <h2>Overview</h2>
  * <p>
@@ -249,7 +251,7 @@ private void completeABatch(boolean toInitialize, boolean needsSpill) {
       tmpBatchesList.add(currentBatch);
       partitionBatchesCount++;
 
-      long batchSize = new RecordBatchSizer(currentBatch).actualSize();
+      long batchSize = new RecordBatchSizer(currentBatch).getActualSize();
       inMemoryBatchStats.add(new HashJoinMemoryCalculator.BatchStat(currentBatch.getRecordCount(), batchSize));
 
       partitionInMemorySize += batchSize;
@@ -498,7 +500,7 @@ public void buildContainersHashTableAndHelper() throws SchemaChangeException {
       for (int recInd = 0; recInd < currentRecordCount; recInd++) {
         int hashCode = HV_vector.getAccessor().get(recInd);
         try {
-          hashTable.put(recInd, htIndex, hashCode);
+          hashTable.put(recInd, htIndex, hashCode, BATCH_SIZE);
         } catch (RetryAfterSpillException RE) {
           throw new OutOfMemoryException("HT put");
         } // Hash Join does not retry
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTable.java
index 194c865ff6..3bf4b86ecf 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTable.java
@@ -82,7 +82,7 @@ void setup(HashTableConfig htConfig, BufferAllocator allocator, VectorContainer
    */
   int getProbeHashCode(int incomingRowIdx) throws SchemaChangeException;
 
-  PutStatus put(int incomingRowIdx, IndexPointer htIdxHolder, int hashCode) throws SchemaChangeException, RetryAfterSpillException;
+  PutStatus put(int incomingRowIdx, IndexPointer htIdxHolder, int hashCode, int batchSize) throws SchemaChangeException, RetryAfterSpillException;
 
   /**
    * @param incomingRowIdx The index of the key in the probe batch.
@@ -130,12 +130,10 @@ void setup(HashTableConfig htConfig, BufferAllocator allocator, VectorContainer
    * Retrieves the key columns and transfers them to the output container. Note this operation removes the key columns from the {@link HashTable}.
    * @param batchIdx The index of a {@link HashTableTemplate.BatchHolder} in the HashTable.
    * @param outContainer The destination container for the key columns.
-   * @param outStartIndex The start index of the key records to transfer.
    * @param numRecords The number of key recorts to transfer.
-   * @param numExpectedRecords
    * @return
    */
-  boolean outputKeys(int batchIdx, VectorContainer outContainer, int outStartIndex, int numRecords, int numExpectedRecords);
+  boolean outputKeys(int batchIdx, VectorContainer outContainer, int numRecords);
 
   /**
    * Returns a message containing memory usage statistics. Intended to be used for printing debugging or error messages.
@@ -148,6 +146,10 @@ void setup(HashTableConfig htConfig, BufferAllocator allocator, VectorContainer
    * @return
    */
   long getActualSize();
+
+  void setTargetBatchRowCount(int batchRowCount);
+
+  int getTargetBatchRowCount();
 }
 
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTableAllocationTracker.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTableAllocationTracker.java
index d72278d81c..7f38ee624f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTableAllocationTracker.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTableAllocationTracker.java
@@ -32,42 +32,35 @@
   }
 
   private final HashTableConfig config;
-  private final int maxBatchHolderSize;
-
   private State state = State.NO_ALLOCATION_IN_PROGRESS;
   private int remainingCapacity;
 
-  protected HashTableAllocationTracker(final HashTableConfig config,
-                                       final int maxBatchHolderSize)
+  protected HashTableAllocationTracker(final HashTableConfig config)
   {
     this.config = Preconditions.checkNotNull(config);
-    this.maxBatchHolderSize = maxBatchHolderSize;
-
     remainingCapacity = config.getInitialCapacity();
   }
 
-  public int getNextBatchHolderSize() {
+  public int getNextBatchHolderSize(int batchSize) {
     state = State.ALLOCATION_IN_PROGRESS;
 
     if (!config.getInitialSizeIsFinal()) {
-      // We don't know the final size of the hash table, so return the default max batch holder size
-      return maxBatchHolderSize;
+      // We don't know the final size of the hash table, so just return the batch size.
+      return batchSize;
     } else {
       // We know the final size of the hash table so we need to compute the next batch holder size.
-
       Preconditions.checkState(remainingCapacity > 0);
-      return computeNextBatchHolderSize();
+      return computeNextBatchHolderSize(batchSize);
     }
   }
 
-  private int computeNextBatchHolderSize() {
-    return Math.min(remainingCapacity, maxBatchHolderSize);
+  private int computeNextBatchHolderSize(int batchSize) {
+    return Math.min(batchSize, remainingCapacity);
   }
 
-  public void commit() {
+  public void commit(int batchSize) {
     Preconditions.checkState(state.equals(State.ALLOCATION_IN_PROGRESS));
-
-    remainingCapacity -= computeNextBatchHolderSize();
+    remainingCapacity -= batchSize;
     state = State.NO_ALLOCATION_IN_PROGRESS;
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTableTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTableTemplate.java
index bb0b1adf2a..756b3f3a20 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTableTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTableTemplate.java
@@ -65,7 +65,9 @@
   // Array of batch holders..each batch holder can hold up to BATCH_SIZE entries
   private ArrayList<BatchHolder> batchHolders;
 
-  private int totalBatchHoldersSize; // the size of all batchHolders
+  private int totalIndexSize; // index size of all batchHolders including current batch
+  private int prevIndexSize; // index size of all batchHolders not including current batch
+  private int currentIndexSize; // prevIndexSize + current batch count.
 
   // Current size of the hash table in terms of number of buckets
   private int tableSize = 0;
@@ -127,13 +129,21 @@
     private IntVector hashValues;
 
     private int maxOccupiedIdx = -1;
-//    private int batchOutputCount = 0;
-
+    private int targetBatchRowCount;
     private int batchIndex = 0;
 
+    public void setTargetBatchRowCount(int targetBatchRowCount) {
+      this.targetBatchRowCount = targetBatchRowCount;
+    }
+
+    public int getTargetBatchRowCount() {
+      return targetBatchRowCount;
+    }
+
     public BatchHolder(int idx, int newBatchHolderSize) {
 
       this.batchIndex = idx;
+      this.targetBatchRowCount = newBatchHolderSize;
 
       htContainer = new VectorContainer();
       boolean success = false;
@@ -152,7 +162,7 @@ public BatchHolder(int idx, int newBatchHolderSize) {
           } else if (vv instanceof VariableWidthVector) {
             long beforeMem = allocator.getAllocatedMemory();
             ((VariableWidthVector) vv).allocateNew(MAX_VARCHAR_SIZE * newBatchHolderSize, newBatchHolderSize);
-            logger.trace("HT allocated {} for varchar of max width {}",allocator.getAllocatedMemory() - beforeMem, MAX_VARCHAR_SIZE);
+            logger.trace("HT allocated {} for varchar of max width {}", allocator.getAllocatedMemory() - beforeMem, MAX_VARCHAR_SIZE);
           } else {
             vv.allocateNew();
           }
@@ -164,7 +174,9 @@ public BatchHolder(int idx, int newBatchHolderSize) {
       } finally {
         if (!success) {
           htContainer.clear();
-          if (links != null) { links.clear();}
+          if (links != null) {
+            links.clear();
+          }
         }
       }
     }
@@ -190,15 +202,14 @@ protected void setup() throws SchemaChangeException {
     private boolean isKeyMatch(int incomingRowIdx,
         IndexPointer currentIdxHolder,
         boolean isProbe) throws SchemaChangeException {
-
       int currentIdxWithinBatch = currentIdxHolder.value & BATCH_MASK;
       boolean match;
 
-      if (currentIdxWithinBatch >= HashTable.BATCH_SIZE) {
-        logger.debug("Batch size = {}, incomingRowIdx = {}, currentIdxWithinBatch = {}.", HashTable.BATCH_SIZE,
-            incomingRowIdx, currentIdxWithinBatch);
+      if (currentIdxWithinBatch >= batchHolders.get((currentIdxHolder.value >>> 16) & BATCH_MASK).getTargetBatchRowCount()) {
+        logger.debug("Batch size = {}, incomingRowIdx = {}, currentIdxWithinBatch = {}.",
+          batchHolders.get((currentIdxHolder.value >>> 16) & BATCH_MASK).getTargetBatchRowCount(), incomingRowIdx, currentIdxWithinBatch);
       }
-      assert (currentIdxWithinBatch < HashTable.BATCH_SIZE);
+      assert (currentIdxWithinBatch < batchHolders.get((currentIdxHolder.value >>> 16) & BATCH_MASK).getTargetBatchRowCount());
       assert (incomingRowIdx < HashTable.BATCH_SIZE);
 
       if (isProbe) {
@@ -217,7 +228,6 @@ private boolean isKeyMatch(int incomingRowIdx,
     // container at the specified index
     private void insertEntry(int incomingRowIdx, int currentIdx, int hashValue, BatchHolder lastEntryBatch, int lastEntryIdxWithinBatch) throws SchemaChangeException {
       int currentIdxWithinBatch = currentIdx & BATCH_MASK;
-
       setValue(incomingRowIdx, currentIdxWithinBatch);
       // setValue may OOM when doubling of one of the VarChar Key Value Vectors
       // This would be caught and retried later (setValue() is idempotent)
@@ -280,8 +290,7 @@ private void rehash(int numbuckets, IntVector newStartIndices, int batchStartIdx
           while (true) {
             if (idx != EMPTY_SLOT) {
               idxWithinBatch = idx & BATCH_MASK;
-              int batchIdx = ((idx >>> 16) & BATCH_MASK);
-              bh = batchHolders.get(batchIdx);
+              bh = batchHolders.get((idx >>> 16) & BATCH_MASK);
             }
 
             if (bh == this && newLinks.getAccessor().get(idxWithinBatch) == EMPTY_SLOT) {
@@ -332,7 +341,7 @@ private void rehash(int numbuckets, IntVector newStartIndices, int batchStartIdx
       hashValues = newHashValues;
     }
 
-    private boolean outputKeys(VectorContainer outContainer, int outStartIndex, int numRecords, int numExpectedRecords) {
+    private boolean outputKeys(VectorContainer outContainer, int numRecords) {
       // set the value count for htContainer's value vectors before the transfer ..
       setValueCount();
 
@@ -344,18 +353,9 @@ private boolean outputKeys(VectorContainer outContainer, int outStartIndex, int
         @SuppressWarnings("resource")
         ValueVector targetVV = outgoingIter.next().getValueVector();
         TransferPair tp = sourceVV.makeTransferPair(targetVV);
-        if ( outStartIndex == 0 && numRecords == numExpectedRecords ) {
-          // The normal case: The whole column key(s) are transfered as is
-          tp.transfer();
-        } else {
-          // Transfer just the required section (does this ever happen ?)
-          // Requires an expensive allocation and copy
-          logger.debug("Performing partial output of keys, from index {}, num {} (out of {})",
-              outStartIndex,numRecords,numExpectedRecords);
-          tp.splitAndTransfer(outStartIndex, numRecords);
-        }
+        // The normal case: The whole column key(s) are transfered as is
+        tp.transfer();
       }
-
       return true;
     }
 
@@ -365,6 +365,7 @@ private void setValueCount() {
         ValueVector vv = vw.getValueVector();
         vv.getMutator().setValueCount(maxOccupiedIdx + 1);
       }
+      htContainer.setRecordCount(maxOccupiedIdx+1);
     }
 
     private void dump(int idx) {
@@ -438,7 +439,7 @@ public long getActualSize() {
         size += ledger.getAccountedSize();
       }
 
-      size += new RecordBatchSizer(htContainer).actualSize();
+      size += new RecordBatchSizer(htContainer).getActualSize();
       return size;
     }
   }
@@ -468,7 +469,7 @@ public void setup(HashTableConfig htConfig, BufferAllocator allocator, VectorCon
     this.incomingProbe = incomingProbe;
     this.outgoing = outgoing;
     this.htContainerOrig = htContainerOrig;
-    this.allocationTracker = new HashTableAllocationTracker(htConfig, BATCH_SIZE);
+    this.allocationTracker = new HashTableAllocationTracker(htConfig);
 
     // round up the initial capacity to nearest highest power of 2
     tableSize = roundUpToPowerOf2(initialCap);
@@ -485,9 +486,12 @@ public void setup(HashTableConfig htConfig, BufferAllocator allocator, VectorCon
 
     // Create the first batch holder
     batchHolders = new ArrayList<BatchHolder>();
-    totalBatchHoldersSize = 0;
     // First BatchHolder is created when the first put request is received.
 
+    prevIndexSize = 0;
+    currentIndexSize = 0;
+    totalIndexSize = 0;
+
     try {
       doSetup(incomingBuild, incomingProbe);
     } catch (SchemaChangeException e) {
@@ -500,7 +504,7 @@ public void setup(HashTableConfig htConfig, BufferAllocator allocator, VectorCon
   @Override
   public void updateInitialCapacity(int initialCapacity) {
     htConfig = htConfig.withInitialCapacity(initialCapacity);
-    allocationTracker = new HashTableAllocationTracker(htConfig, BATCH_SIZE);
+    allocationTracker = new HashTableAllocationTracker(htConfig);
     enlargeEmptyHashTableIfNeeded(initialCapacity);
   }
 
@@ -547,7 +551,9 @@ public void clear() {
       }
       batchHolders.clear();
       batchHolders = null;
-      totalBatchHoldersSize = 0;
+      prevIndexSize = 0;
+      currentIndexSize = 0;
+      totalIndexSize = 0;
     }
     startIndices.clear();
     // currentIdxHolder = null; // keep IndexPointer in case HT is reused
@@ -573,10 +579,15 @@ private void retryAfterOOM(boolean batchAdded) throws RetryAfterSpillException {
     if ( batchAdded ) {
       logger.trace("OOM - Removing index {} from the batch holders list",batchHolders.size() - 1);
       BatchHolder bh = batchHolders.remove(batchHolders.size() - 1);
-      totalBatchHoldersSize -= BATCH_SIZE;
+      prevIndexSize = batchHolders.size() > 1 ? (batchHolders.size()-1) * BATCH_SIZE : 0;
+      currentIndexSize = prevIndexSize + (batchHolders.size() > 0 ? batchHolders.get(batchHolders.size()-1).getTargetBatchRowCount() : 0);
+      totalIndexSize = batchHolders.size() * BATCH_SIZE;
+      // update freeIndex to point to end of last batch + 1
+      freeIndex = totalIndexSize + 1;
       bh.clear();
+    } else {
+      freeIndex--;
     }
-    freeIndex--;
     throw new RetryAfterSpillException();
   }
 
@@ -618,7 +629,7 @@ public int getProbeHashCode(int incomingRowIdx) throws SchemaChangeException {
    * @return Status - the key(s) was ADDED or was already PRESENT
    */
   @Override
-  public PutStatus put(int incomingRowIdx, IndexPointer htIdxHolder, int hashCode) throws SchemaChangeException, RetryAfterSpillException {
+  public PutStatus put(int incomingRowIdx, IndexPointer htIdxHolder, int hashCode, int targetBatchRowCount) throws SchemaChangeException, RetryAfterSpillException {
 
     int bucketIndex = getBucketIndex(hashCode, numBuckets());
     int startIdx = startIndices.getAccessor().get(bucketIndex);
@@ -633,7 +644,7 @@ public PutStatus put(int incomingRowIdx, IndexPointer htIdxHolder, int hashCode)
           /* isKeyMatch() below also advances the currentIdxHolder to the next link */) {
 
       // remember the current link, which would be the last when the next link is empty
-      lastEntryBatch = batchHolders.get((currentIdxHolder.value >>> 16) & HashTable.BATCH_MASK);
+      lastEntryBatch = batchHolders.get((currentIdxHolder.value >>> 16) & BATCH_MASK);
       lastEntryIdxWithinBatch = currentIdxHolder.value & BATCH_MASK;
 
       if (lastEntryBatch.isKeyMatch(incomingRowIdx, currentIdxHolder, false)) {
@@ -646,14 +657,18 @@ public PutStatus put(int incomingRowIdx, IndexPointer htIdxHolder, int hashCode)
     currentIdx = freeIndex++;
     boolean addedBatch = false;
     try {  // ADD A BATCH
-      addedBatch = addBatchIfNeeded(currentIdx);
+      addedBatch = addBatchIfNeeded(currentIdx, targetBatchRowCount);
+      if (addedBatch) {
+        // If we just added the batch, update the current index to point to beginning of new batch.
+        currentIdx = (batchHolders.size() - 1) * BATCH_SIZE;
+        freeIndex = currentIdx + 1;
+      }
     } catch (OutOfMemoryException OOME) {
-      retryAfterOOM( currentIdx < batchHolders.size() * BATCH_SIZE );
+      retryAfterOOM( currentIdx < totalIndexSize);
     }
 
     try { // INSERT ENTRY
       BatchHolder bh = batchHolders.get((currentIdx >>> 16) & BATCH_MASK);
-
       bh.insertEntry(incomingRowIdx, currentIdx, hashCode, lastEntryBatch, lastEntryIdxWithinBatch);
       numEntries++;
     } catch (OutOfMemoryException OOME) { retryAfterOOM( addedBatch ); }
@@ -683,7 +698,7 @@ public PutStatus put(int incomingRowIdx, IndexPointer htIdxHolder, int hashCode)
     }
     htIdxHolder.value = currentIdx;
     return  addedBatch ? PutStatus.NEW_BATCH_ADDED :
-        ( freeIndex + 1 > totalBatchHoldersSize /* batchHolders.size() * BATCH_SIZE */ ) ?
+        (freeIndex + 1 > currentIndexSize) ?
         PutStatus.KEY_ADDED_LAST : // the last key in the batch
         PutStatus.KEY_ADDED;     // otherwise
   }
@@ -715,20 +730,22 @@ public int probeForKey(int incomingRowIdx, int hashCode) throws SchemaChangeExce
   // Add a new BatchHolder to the list of batch holders if needed. This is based on the supplied
   // currentIdx; since each BatchHolder can hold up to BATCH_SIZE entries, if the currentIdx exceeds
   // the capacity, we will add a new BatchHolder. Return true if a new batch was added.
-  private boolean addBatchIfNeeded(int currentIdx) throws SchemaChangeException {
-    // int totalBatchSize = batchHolders.size() * BATCH_SIZE;
-
-    if (currentIdx >= totalBatchHoldersSize) {
-      BatchHolder bh = newBatchHolder(batchHolders.size(), allocationTracker.getNextBatchHolderSize());
+  private boolean addBatchIfNeeded(int currentIdx, int batchRowCount) throws SchemaChangeException {
+     // Add a new batch if this is the first batch or
+     // index is greater than current batch target count i.e. we reached the limit of current batch.
+     if (batchHolders.size() == 0 || (currentIdx >= currentIndexSize)) {
+      final int allocationSize = allocationTracker.getNextBatchHolderSize(batchRowCount);
+      final BatchHolder bh = newBatchHolder(batchHolders.size(), allocationSize);
       batchHolders.add(bh);
+      prevIndexSize = batchHolders.size() > 1 ? (batchHolders.size()-1)*BATCH_SIZE : 0;
+      currentIndexSize = prevIndexSize + batchHolders.get(batchHolders.size()-1).getTargetBatchRowCount();
+      totalIndexSize = batchHolders.size() * BATCH_SIZE;
       bh.setup();
       if (EXTRA_DEBUG) {
         logger.debug("HashTable: Added new batch. Num batches = {}.", batchHolders.size());
       }
 
-      allocationTracker.commit();
-
-      totalBatchHoldersSize += BATCH_SIZE; // total increased by 1 batch
+      allocationTracker.commit(allocationSize);
       return true;
     }
     return false;
@@ -781,10 +798,12 @@ private void resizeAndRehashIfNeeded() {
 
     IntVector newStartIndices = allocMetadataVector(tableSize, EMPTY_SLOT);
 
+    int idx = 0;
     for (int i = 0; i < batchHolders.size(); i++) {
       BatchHolder bh = batchHolders.get(i);
-      int batchStartIdx = i * BATCH_SIZE;
+      int batchStartIdx = idx;
       bh.rehash(tableSize, newStartIndices, batchStartIdx);
+      idx += bh.getTargetBatchRowCount();
     }
 
     startIndices.clear();
@@ -795,8 +814,8 @@ private void resizeAndRehashIfNeeded() {
       logger.debug("Number of buckets = {}.", startIndices.getAccessor().getValueCount());
       for (int i = 0; i < startIndices.getAccessor().getValueCount(); i++) {
         logger.debug("Bucket: {}, startIdx[ {} ] = {}.", i, i, startIndices.getAccessor().get(i));
-        int idx = startIndices.getAccessor().get(i);
-        BatchHolder bh = batchHolders.get((idx >>> 16) & BATCH_MASK);
+        int startIdx = startIndices.getAccessor().get(i);
+        BatchHolder bh = batchHolders.get((startIdx >>> 16) & BATCH_MASK);
         bh.dump(idx);
       }
     }
@@ -830,7 +849,9 @@ public void reset() {
     freeIndex = 0; // all batch holders are gone
     // reallocate batch holders, and the hash table to the original size
     batchHolders = new ArrayList<BatchHolder>();
-    totalBatchHoldersSize = 0;
+    prevIndexSize = 0;
+    currentIndexSize = 0;
+    totalIndexSize = 0;
     startIndices = allocMetadataVector(originalTableSize, EMPTY_SLOT);
   }
   public void updateIncoming(VectorContainer newIncoming, RecordBatch newIncomingProbe) {
@@ -845,9 +866,9 @@ public void updateIncoming(VectorContainer newIncoming, RecordBatch newIncomingP
   }
 
   @Override
-  public boolean outputKeys(int batchIdx, VectorContainer outContainer, int outStartIndex, int numRecords, int numExpectedRecords) {
+  public boolean outputKeys(int batchIdx, VectorContainer outContainer, int numRecords) {
     assert batchIdx < batchHolders.size();
-    return batchHolders.get(batchIdx).outputKeys(outContainer, outStartIndex, numRecords, numExpectedRecords);
+    return batchHolders.get(batchIdx).outputKeys(outContainer, numRecords);
   }
 
   private IntVector allocMetadataVector(int size, int initialValue) {
@@ -890,4 +911,14 @@ public String makeDebugString() {
     return String.format("[numBuckets = %d, numEntries = %d, numBatchHolders = %d, actualSize = %s]",
       numBuckets(), numEntries, batchHolders.size(), HashJoinMemoryCalculator.PartitionStatSet.prettyPrintBytes(getActualSize()));
   }
+
+  @Override
+  public void setTargetBatchRowCount(int batchRowCount) {
+    batchHolders.get(batchHolders.size()-1).targetBatchRowCount = batchRowCount;
+  }
+
+  @Override
+  public int getTargetBatchRowCount() {
+    return batchHolders.get(batchHolders.size()-1).targetBatchRowCount;
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/flatten/FlattenRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/flatten/FlattenRecordBatch.java
index be44c94c0a..8421d6ee09 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/flatten/FlattenRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/flatten/FlattenRecordBatch.java
@@ -136,7 +136,7 @@ public void update() {
       final int avgRowWidthFlattenColumn = columnSize.getNetSizePerEntry();
 
       // Average rowWidth excluding the flatten column.
-      final int avgRowWidthWithOutFlattenColumn = getRecordBatchSizer().netRowWidth() - avgRowWidthFlattenColumn;
+      final int avgRowWidthWithOutFlattenColumn = getRecordBatchSizer().getNetRowWidth() - avgRowWidthFlattenColumn;
 
       // Average rowWidth of single element in the flatten list.
       // subtract the offset vector size from column data size.
@@ -157,9 +157,7 @@ public void update() {
       // i.e. all rows fit within memory budget.
       setOutputRowCount(Math.min(columnSize.getElementCount(), getOutputRowCount()));
 
-      if (logger.isDebugEnabled()) {
-        logger.debug("BATCH_STATS, incoming:\n {}", getRecordBatchSizer());
-      }
+      logger.debug("BATCH_STATS, incoming: {}", getRecordBatchSizer());
 
       updateIncomingStats();
     }
@@ -171,6 +169,8 @@ public FlattenRecordBatch(FlattenPOP pop, RecordBatch incoming, FragmentContext
     // get the output batch size from config.
     int configuredBatchSize = (int) context.getOptions().getOption(ExecConstants.OUTPUT_BATCH_SIZE_VALIDATOR);
     flattenMemoryManager = new FlattenMemoryManager(configuredBatchSize);
+
+    logger.debug("BATCH_STATS, configured output batch size: {}", configuredBatchSize);
   }
 
   @Override
@@ -263,7 +263,7 @@ protected IterOutcome doWork() {
     flattenMemoryManager.updateOutgoingStats(outputRecords);
 
     if (logger.isDebugEnabled()) {
-      logger.debug("BATCH_STATS, outgoing:\n {}", new RecordBatchSizer(this));
+      logger.debug("BATCH_STATS, outgoing: {}", new RecordBatchSizer(this));
     }
 
     // Get the final outcome based on hasRemainder since that will determine if all the incoming records were
@@ -516,14 +516,15 @@ private void updateStats() {
     stats.setLongStat(Metric.AVG_OUTPUT_ROW_BYTES, flattenMemoryManager.getAvgOutputRowWidth());
     stats.setLongStat(Metric.OUTPUT_RECORD_COUNT, flattenMemoryManager.getTotalOutputRecords());
 
-    logger.debug("BATCH_STATS, incoming aggregate: count : {}, avg bytes : {},  avg row bytes : {}, record count : {}",
-      flattenMemoryManager.getNumIncomingBatches(), flattenMemoryManager.getAvgInputBatchSize(),
-      flattenMemoryManager.getAvgInputRowWidth(), flattenMemoryManager.getTotalInputRecords());
-
-    logger.debug("BATCH_STATS, outgoing aggregate: count : {}, avg bytes : {},  avg row bytes : {}, record count : {}",
-      flattenMemoryManager.getNumOutgoingBatches(), flattenMemoryManager.getAvgOutputBatchSize(),
-      flattenMemoryManager.getAvgOutputRowWidth(), flattenMemoryManager.getTotalOutputRecords());
+    if (logger.isDebugEnabled()) {
+      logger.debug("BATCH_STATS, incoming aggregate: count : {}, avg bytes : {},  avg row bytes : {}, record count : {}",
+        flattenMemoryManager.getNumIncomingBatches(), flattenMemoryManager.getAvgInputBatchSize(),
+        flattenMemoryManager.getAvgInputRowWidth(), flattenMemoryManager.getTotalInputRecords());
 
+      logger.debug("BATCH_STATS, outgoing aggregate: count : {}, avg bytes : {},  avg row bytes : {}, record count : {}",
+        flattenMemoryManager.getNumOutgoingBatches(), flattenMemoryManager.getAvgOutputBatchSize(),
+        flattenMemoryManager.getAvgOutputRowWidth(), flattenMemoryManager.getTotalOutputRecords());
+    }
   }
 
   @Override
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
index ee7a8a38c1..345d18200d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
@@ -19,6 +19,7 @@
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
@@ -57,13 +58,13 @@
 import org.apache.drill.exec.physical.impl.spill.SpillSet;
 import org.apache.drill.exec.record.AbstractBinaryRecordBatch;
 import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.JoinBatchMemoryManager;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.RecordBatchSizer;
 import org.apache.drill.exec.record.VectorWrapper;
-import org.apache.drill.exec.vector.FixedWidthVector;
 import org.apache.drill.exec.vector.IntVector;
 import org.apache.drill.exec.vector.ValueVector;
-import org.apache.drill.exec.vector.VariableWidthVector;
 import org.apache.drill.exec.vector.complex.AbstractContainerVector;
 import org.apache.calcite.rel.core.JoinRelType;
 
@@ -95,11 +96,6 @@
    */
   private int RECORDS_PER_BATCH; // internal batches
 
-  /**
-   * The maximum number of records in each outgoing batch.
-   */
-  private static final int TARGET_RECORDS_PER_BATCH = 4000;
-
   // Join type, INNER, LEFT, RIGHT or OUTER
   private final JoinRelType joinType;
 
@@ -172,7 +168,8 @@
     public String outerSpillFile;
     int cycleNum;
     int origPartn;
-    int prevOrigPartn; }
+    int prevOrigPartn;
+  }
 
   /**
    * Queue of spilled partitions to process.
@@ -181,7 +178,6 @@
   private HJSpilledPartition spilledInners[]; // for the outer to find the partition
 
   public enum Metric implements MetricDef {
-
     NUM_BUCKETS,
     NUM_ENTRIES,
     NUM_RESIZING,
@@ -190,8 +186,19 @@
     SPILLED_PARTITIONS, // number of original partitions spilled to disk
     SPILL_MB,         // Number of MB of data spilled to disk. This amount is first written,
                       // then later re-read. So, disk I/O is twice this amount.
-    SPILL_CYCLE       // 0 - no spill, 1 - spill, 2 - SECONDARY, 3 - TERTIARY
-    ;
+    SPILL_CYCLE,       // 0 - no spill, 1 - spill, 2 - SECONDARY, 3 - TERTIARY
+    LEFT_INPUT_BATCH_COUNT,
+    LEFT_AVG_INPUT_BATCH_BYTES,
+    LEFT_AVG_INPUT_ROW_BYTES,
+    LEFT_INPUT_RECORD_COUNT,
+    RIGHT_INPUT_BATCH_COUNT,
+    RIGHT_AVG_INPUT_BATCH_BYTES,
+    RIGHT_AVG_INPUT_ROW_BYTES,
+    RIGHT_INPUT_RECORD_COUNT,
+    OUTPUT_BATCH_COUNT,
+    AVG_OUTPUT_BATCH_BYTES,
+    AVG_OUTPUT_ROW_BYTES,
+    OUTPUT_RECORD_COUNT;
 
     // duplicate for hash ag
 
@@ -221,12 +228,7 @@ protected void buildSchema() throws SchemaChangeException {
       throw new SchemaChangeException(e);
     }
 
-    // Build the container schema and set the counts
-    for (final VectorWrapper<?> w : container) {
-      w.getValueVector().allocateNew();
-    }
     container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
-    container.setRecordCount(outputRecords);
   }
 
   @Override
@@ -234,6 +236,13 @@ protected boolean prefetchFirstBatchFromBothSides() {
     leftUpstream = sniffNonEmptyBatch(0, left);
     rightUpstream = sniffNonEmptyBatch(1, right);
 
+    // For build side, use aggregate i.e. average row width across batches
+    batchMemoryManager.update(LEFT_INDEX, 0);
+    batchMemoryManager.update(RIGHT_INDEX, 0, true);
+
+    logger.debug("BATCH_STATS, incoming left: {}", batchMemoryManager.getRecordBatchSizer(LEFT_INDEX));
+    logger.debug("BATCH_STATS, incoming right: {}", batchMemoryManager.getRecordBatchSizer(RIGHT_INDEX));
+
     if (leftUpstream == IterOutcome.STOP || rightUpstream == IterOutcome.STOP) {
       state = BatchState.STOP;
       return false;
@@ -333,10 +342,21 @@ public IterOutcome innerNext() {
            joinType != JoinRelType.INNER) {  // or if this is a left/full outer join
 
         // Allocate the memory for the vectors in the output container
-        allocateVectors();
+        batchMemoryManager.allocateVectors(container);
+        hashJoinProbe.setTargetOutputCount(batchMemoryManager.getOutputRowCount());
 
         outputRecords = hashJoinProbe.probeAndProject();
 
+        for (final VectorWrapper<?> v : container) {
+          v.getValueVector().getMutator().setValueCount(outputRecords);
+        }
+        container.setRecordCount(outputRecords);
+
+        batchMemoryManager.updateOutgoingStats(outputRecords);
+        if (logger.isDebugEnabled()) {
+          logger.debug("BATCH_STATS, outgoing: {}", new RecordBatchSizer(this));
+        }
+
         /* We are here because of one the following
          * 1. Completed processing of all the records and we are done
          * 2. We've filled up the outgoing batch to the maximum and we need to return upstream
@@ -347,10 +367,6 @@ public IterOutcome innerNext() {
             state = BatchState.NOT_FIRST;
           }
 
-          for (final VectorWrapper<?> v : container) {
-            v.getValueVector().getMutator().setValueCount(outputRecords);
-          }
-
           return IterOutcome.OK;
         }
 
@@ -557,7 +573,8 @@ private void initializeBuild() {
         RECORDS_PER_BATCH,
         maxBatchSize,
         maxBatchSize,
-        TARGET_RECORDS_PER_BATCH,
+        batchMemoryManager.getOutputRowCount(),
+        batchMemoryManager.getOutputBatchSize(),
         HashTable.DEFAULT_LOAD_FACTOR);
 
       disableSpilling(null);
@@ -628,7 +645,8 @@ public void executeBuildPhase() throws SchemaChangeException {
         RECORDS_PER_BATCH,
         maxBatchSize,
         maxBatchSize,
-        TARGET_RECORDS_PER_BATCH,
+        batchMemoryManager.getOutputRowCount(),
+        batchMemoryManager.getOutputBatchSize(),
         HashTable.DEFAULT_LOAD_FACTOR);
 
       if (firstCycle && doMemoryCalculation) {
@@ -665,6 +683,7 @@ public void executeBuildPhase() throws SchemaChangeException {
         for (HashPartition partn : partitions) { partn.updateBatches(); }
         // Fall through
       case OK:
+        batchMemoryManager.update(buildBatch, RIGHT_INDEX, 0, true);
         // Special treatment (when no spill, and single partition) -- use the incoming vectors as they are (no row copy)
         if ( numPartitions == 1 ) {
           partitions[0].appendBatch(buildBatch);
@@ -803,22 +822,6 @@ private void setupOutputContainerSchema() {
 
   }
 
-  private void allocateVectors() {
-    for (final VectorWrapper<?> vectorWrapper : container) {
-      ValueVector valueVector = vectorWrapper.getValueVector();
-
-      if (valueVector instanceof FixedWidthVector) {
-        ((FixedWidthVector) valueVector).allocateNew(TARGET_RECORDS_PER_BATCH);
-      } else if (valueVector instanceof VariableWidthVector) {
-        ((VariableWidthVector) valueVector).allocateNew(8 * TARGET_RECORDS_PER_BATCH, TARGET_RECORDS_PER_BATCH);
-      } else {
-        valueVector.allocateNew();
-      }
-    }
-
-    container.setRecordCount(0); // reset container's counter back to zero records
-  }
-
   // (After the inner side was read whole) - Has that inner partition spilled
   public boolean isSpilledInner(int part) {
     if ( spilledInners == null ) { return false; } // empty inner
@@ -879,6 +882,16 @@ public HashJoinBatch(HashJoinPOP popConfig, FragmentContext context,
 
     // Create empty partitions (in the ctor - covers the case where right side is empty)
     partitions = new HashPartition[0];
+
+    // get the output batch size from config.
+    final int configuredBatchSize = (int) context.getOptions().getOption(ExecConstants.OUTPUT_BATCH_SIZE_VALIDATOR);
+    final double avail_mem_factor = (double) context.getOptions().getOption(ExecConstants.OUTPUT_BATCH_SIZE_AVAIL_MEM_FACTOR_VALIDATOR);
+    int outputBatchSize = Math.min(configuredBatchSize, Integer.highestOneBit((int)(allocator.getLimit() * avail_mem_factor)));
+    logger.debug("BATCH_STATS, configured output batch size: {}, allocated memory {}, avail mem factor {}, output batch size: {}",
+      configuredBatchSize, allocator.getLimit(), avail_mem_factor, outputBatchSize);
+
+    batchMemoryManager = new JoinBatchMemoryManager(outputBatchSize, left, right, new HashSet<>());
+    logger.debug("BATCH_STATS, configured output batch size: {}", configuredBatchSize);
   }
 
   /**
@@ -966,6 +979,23 @@ public void killIncoming(boolean sendUpstream) {
     buildBatch.kill(sendUpstream);
   }
 
+  public void updateMetrics() {
+    stats.setLongStat(HashJoinBatch.Metric.LEFT_INPUT_BATCH_COUNT, batchMemoryManager.getNumIncomingBatches(LEFT_INDEX));
+    stats.setLongStat(HashJoinBatch.Metric.LEFT_AVG_INPUT_BATCH_BYTES, batchMemoryManager.getAvgInputBatchSize(LEFT_INDEX));
+    stats.setLongStat(HashJoinBatch.Metric.LEFT_AVG_INPUT_ROW_BYTES, batchMemoryManager.getAvgInputRowWidth(LEFT_INDEX));
+    stats.setLongStat(HashJoinBatch.Metric.LEFT_INPUT_RECORD_COUNT, batchMemoryManager.getTotalInputRecords(LEFT_INDEX));
+
+    stats.setLongStat(HashJoinBatch.Metric.RIGHT_INPUT_BATCH_COUNT, batchMemoryManager.getNumIncomingBatches(RIGHT_INDEX));
+    stats.setLongStat(HashJoinBatch.Metric.RIGHT_AVG_INPUT_BATCH_BYTES, batchMemoryManager.getAvgInputBatchSize(RIGHT_INDEX));
+    stats.setLongStat(HashJoinBatch.Metric.RIGHT_AVG_INPUT_ROW_BYTES, batchMemoryManager.getAvgInputRowWidth(RIGHT_INDEX));
+    stats.setLongStat(HashJoinBatch.Metric.RIGHT_INPUT_RECORD_COUNT, batchMemoryManager.getTotalInputRecords(RIGHT_INDEX));
+
+    stats.setLongStat(HashJoinBatch.Metric.OUTPUT_BATCH_COUNT, batchMemoryManager.getNumOutgoingBatches());
+    stats.setLongStat(HashJoinBatch.Metric.AVG_OUTPUT_BATCH_BYTES, batchMemoryManager.getAvgOutputBatchSize());
+    stats.setLongStat(HashJoinBatch.Metric.AVG_OUTPUT_ROW_BYTES, batchMemoryManager.getAvgOutputRowWidth());
+    stats.setLongStat(HashJoinBatch.Metric.OUTPUT_RECORD_COUNT, batchMemoryManager.getTotalOutputRecords());
+  }
+
   @Override
   public void close() {
     if ( cycleNum > 0 ) { // spilling happened
@@ -973,6 +1003,23 @@ public void close() {
       // SpilledRecordBatch "scanners" as it only knows about the original left/right ops.
       killIncoming(false);
     }
+
+    updateMetrics();
+
+    if (logger.isDebugEnabled()) {
+      logger.debug("BATCH_STATS, incoming aggregate left: batch count : {}, avg bytes : {},  avg row bytes : {}, record count : {}",
+        batchMemoryManager.getNumIncomingBatches(JoinBatchMemoryManager.LEFT_INDEX), batchMemoryManager.getAvgInputBatchSize(JoinBatchMemoryManager.LEFT_INDEX),
+        batchMemoryManager.getAvgInputRowWidth(JoinBatchMemoryManager.LEFT_INDEX), batchMemoryManager.getTotalInputRecords(JoinBatchMemoryManager.LEFT_INDEX));
+
+      logger.debug("BATCH_STATS, incoming aggregate right: batch count : {}, avg bytes : {},  avg row bytes : {}, record count : {}",
+        batchMemoryManager.getNumIncomingBatches(JoinBatchMemoryManager.RIGHT_INDEX), batchMemoryManager.getAvgInputBatchSize(JoinBatchMemoryManager.RIGHT_INDEX),
+        batchMemoryManager.getAvgInputRowWidth(JoinBatchMemoryManager.RIGHT_INDEX), batchMemoryManager.getTotalInputRecords(JoinBatchMemoryManager.RIGHT_INDEX));
+
+      logger.debug("BATCH_STATS, outgoing aggregate: batch count : {}, avg bytes : {},  avg row bytes : {}, record count : {}",
+        batchMemoryManager.getNumOutgoingBatches(), batchMemoryManager.getAvgOutputBatchSize(),
+        batchMemoryManager.getAvgOutputRowWidth(), batchMemoryManager.getTotalOutputRecords());
+    }
+
     this.cleanup();
     super.close();
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinMechanicalMemoryCalculator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinMechanicalMemoryCalculator.java
index 618e80e13b..fb087a0fd0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinMechanicalMemoryCalculator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinMechanicalMemoryCalculator.java
@@ -77,6 +77,7 @@ public void initialize(boolean autoTune,
                            int maxBatchNumRecordsBuild,
                            int maxBatchNumRecordsProbe,
                            int outputBatchNumRecords,
+                           int outputBatchSize,
                            double loadFactor) {
       this.initialPartitions = initialPartitions;
     }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinMemoryCalculator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinMemoryCalculator.java
index 71292a550b..868fbfd10b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinMemoryCalculator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinMemoryCalculator.java
@@ -100,6 +100,7 @@ void initialize(boolean autoTune,
                     int maxBatchNumRecordsBuild,
                     int maxBatchNumRecordsProbe,
                     int outputBatchNumRecords,
+                    int outputBatchSize,
                     double loadFactor);
 
     void setPartitionStatSet(PartitionStatSet partitionStatSet);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinMemoryCalculatorImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinMemoryCalculatorImpl.java
index ed0adc5d81..37f33295ee 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinMemoryCalculatorImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinMemoryCalculatorImpl.java
@@ -142,6 +142,7 @@ public void initialize(boolean autoTune,
                            int maxBatchNumRecordsBuild,
                            int maxBatchNumRecordsProbe,
                            int outputBatchNumRecords,
+                           int outputBatchSize,
                            double loadFactor) {
       this.initialPartitions = initialPartitions;
     }
@@ -203,7 +204,7 @@ public HashJoinState getState() {
    * <h1>Life Cycle</h1>
    * <p>
    *   <ul>
-   *     <li><b>Step 0:</b> Call {@link #initialize(boolean, boolean, RecordBatch, RecordBatch, Set, long, int, int, int, int, int, int, double)}.
+   *     <li><b>Step 0:</b> Call {@link #initialize(boolean, boolean, RecordBatch, RecordBatch, Set, long, int, int, int, int, int, int, int, double)}.
    *     This will initialize the StateCalculate with the additional information it needs.</li>
    *     <li><b>Step 1:</b> Call {@link #getNumPartitions()} to see the number of partitions that fit in memory.</li>
    *     <li><b>Step 2:</b> Call {@link #shouldSpill()} To determine if spilling needs to occurr.</li>
@@ -233,9 +234,7 @@ public HashJoinState getState() {
     private int partitions;
     private int recordsPerPartitionBatchBuild;
     private int recordsPerPartitionBatchProbe;
-    private int outputBatchNumRecords;
-    private Map<String, Long> buildValueSizes;
-    private Map<String, Long> probeValueSizes;
+    private int outputBatchSize;
     private Map<String, Long> keySizes;
     private boolean autoTune;
     private boolean reserveHash;
@@ -273,6 +272,7 @@ public void initialize(boolean autoTune,
                            int maxBatchNumRecordsBuild,
                            int maxBatchNumRecordsProbe,
                            int outputBatchNumRecords,
+                           int outputBatchSize,
                            double loadFactor) {
       Preconditions.checkNotNull(buildSideBatch);
       Preconditions.checkNotNull(probeSideBatch);
@@ -300,8 +300,6 @@ public void initialize(boolean autoTune,
 
       initialize(autoTune,
         reserveHash,
-        buildValueSizes,
-        probeValueSizes,
         keySizes,
         memoryAvailable,
         initialPartitions,
@@ -313,7 +311,7 @@ public void initialize(boolean autoTune,
         recordsPerPartitionBatchProbe,
         maxBatchNumRecordsBuild,
         maxBatchNumRecordsProbe,
-        outputBatchNumRecords,
+        outputBatchSize,
         loadFactor);
     }
 
@@ -352,8 +350,6 @@ public static long getBatchSizeEstimate(final RecordBatch recordBatch) {
     @VisibleForTesting
     protected void initialize(boolean autoTune,
                               boolean reserveHash,
-                              CaseInsensitiveMap<Long> buildValueSizes,
-                              CaseInsensitiveMap<Long> probeValueSizes,
                               CaseInsensitiveMap<Long> keySizes,
                               long memoryAvailable,
                               int initialPartitions,
@@ -365,7 +361,7 @@ protected void initialize(boolean autoTune,
                               int recordsPerPartitionBatchProbe,
                               int maxBatchNumRecordsBuild,
                               int maxBatchNumRecordsProbe,
-                              int outputBatchNumRecords,
+                              int outputBatchSize,
                               double loadFactor) {
       Preconditions.checkState(!firstInitialized);
       Preconditions.checkArgument(initialPartitions >= 1);
@@ -374,8 +370,6 @@ protected void initialize(boolean autoTune,
       this.loadFactor = loadFactor;
       this.autoTune = autoTune;
       this.reserveHash = reserveHash;
-      this.buildValueSizes = Preconditions.checkNotNull(buildValueSizes);
-      this.probeValueSizes = Preconditions.checkNotNull(probeValueSizes);
       this.keySizes = Preconditions.checkNotNull(keySizes);
       this.memoryAvailable = memoryAvailable;
       this.buildBatchSize = buildBatchSize;
@@ -387,7 +381,7 @@ protected void initialize(boolean autoTune,
       this.recordsPerPartitionBatchProbe = recordsPerPartitionBatchProbe;
       this.maxBatchNumRecordsBuild = maxBatchNumRecordsBuild;
       this.maxBatchNumRecordsProbe = maxBatchNumRecordsProbe;
-      this.outputBatchNumRecords = outputBatchNumRecords;
+      this.outputBatchSize = outputBatchSize;
 
       calculateMemoryUsage();
 
@@ -448,8 +442,7 @@ private void calculateMemoryUsage()
         safetyFactor,
         reserveHash);
 
-      maxOutputBatchSize = computeMaxOutputBatchSize(buildValueSizes, probeValueSizes, keySizes,
-        outputBatchNumRecords, safetyFactor, fragmentationFactor);
+      maxOutputBatchSize = (long) ((double)outputBatchSize * fragmentationFactor * safetyFactor);
 
       long probeReservedMemory;
 
@@ -519,18 +512,6 @@ private void calculateMemoryUsage()
       }
     }
 
-    public static long computeMaxOutputBatchSize(Map<String, Long> buildValueSizes,
-                                                 Map<String, Long> probeValueSizes,
-                                                 Map<String, Long> keySizes,
-                                                 int outputNumRecords,
-                                                 double safetyFactor,
-                                                 double fragmentationFactor) {
-      long outputSize = HashTableSizeCalculatorConservativeImpl.computeVectorSizes(keySizes, outputNumRecords, safetyFactor)
-        + HashTableSizeCalculatorConservativeImpl.computeVectorSizes(buildValueSizes, outputNumRecords, safetyFactor)
-        + HashTableSizeCalculatorConservativeImpl.computeVectorSizes(probeValueSizes, outputNumRecords, safetyFactor);
-      return RecordBatchSizer.multiplyByFactor(outputSize, fragmentationFactor);
-    }
-
     @Override
     public boolean shouldSpill() {
       Preconditions.checkState(initialized);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinProbe.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinProbe.java
index f212605d5d..5059b18d21 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinProbe.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinProbe.java
@@ -42,4 +42,6 @@
   void setupHashJoinProbe(RecordBatch probeBatch, HashJoinBatch outgoing, JoinRelType joinRelType, RecordBatch.IterOutcome leftStartState, HashPartition[] partitions, int cycleNum, VectorContainer container, HashJoinBatch.HJSpilledPartition[] spilledInners, boolean buildSideIsEmpty, int numPartitions, int rightHVColPosition);
   int  probeAndProject() throws SchemaChangeException;
   void changeToFinalProbeState();
+  void setTargetOutputCount(int targetOutputCount);
+  int getOutputCount();
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinProbeTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinProbeTemplate.java
index 75c3073652..46f2fa3690 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinProbeTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinProbeTemplate.java
@@ -31,6 +31,8 @@
 import org.apache.drill.exec.vector.IntVector;
 import org.apache.drill.exec.vector.ValueVector;
 
+import static org.apache.drill.exec.record.JoinBatchMemoryManager.LEFT_INDEX;
+
 public abstract class HashJoinProbeTemplate implements HashJoinProbe {
 
   VectorContainer container; // the outgoing container
@@ -45,8 +47,6 @@
 
   private HashJoinBatch outgoingJoinBatch = null;
 
-  private static final int TARGET_RECORDS_PER_BATCH = 4000;
-
   // Number of records to process on the probe side
   private int recordsToProcess = 0;
 
@@ -83,6 +83,16 @@
   private int partitionMask = 0; // numPartitions - 1
   private int bitsInMask = 0; // number of bits in the MASK
   private int rightHVColPosition;
+  private int targetOutputRecords;
+
+  @Override
+  public void setTargetOutputCount(int targetOutputRecords) {
+    this.targetOutputRecords = targetOutputRecords;
+  }
+
+  public int getOutputCount() {
+    return outputRecords;
+  }
 
   /**
    *  Setup the Hash Join Probe object
@@ -209,7 +219,7 @@ private int outputOuterRow(VectorContainer probeSrcContainer, int probeSrcIndex,
 
 
   private void executeProjectRightPhase(int currBuildPart) {
-    while (outputRecords < TARGET_RECORDS_PER_BATCH && recordsProcessed < recordsToProcess) {
+    while (outputRecords < targetOutputRecords && recordsProcessed < recordsToProcess) {
       outputRecords =
         outputRow(partitions[currBuildPart].getContainers(), unmatchedBuildIndexes.get(recordsProcessed),
           null /* no probeBatch */, 0 /* no probe index */ );
@@ -219,7 +229,7 @@ private void executeProjectRightPhase(int currBuildPart) {
 
   private void executeProbePhase() throws SchemaChangeException {
 
-    while (outputRecords < TARGET_RECORDS_PER_BATCH && probeState != ProbeState.DONE && probeState != ProbeState.PROJECT_RIGHT) {
+    while (outputRecords < targetOutputRecords && probeState != ProbeState.DONE && probeState != ProbeState.PROJECT_RIGHT) {
 
       // Check if we have processed all records in this batch we need to invoke next
       if (recordsProcessed == recordsToProcess) {
@@ -262,6 +272,7 @@ private void executeProbePhase() throws SchemaChangeException {
                 probeBatch.getSchema());
             }
           case OK:
+            setTargetOutputCount(outgoingJoinBatch.getBatchMemoryManager().update(probeBatch, LEFT_INDEX,outputRecords));
             recordsToProcess = probeBatch.getRecordCount();
             recordsProcessed = 0;
             // If we received an empty batch do nothing
@@ -274,10 +285,9 @@ private void executeProbePhase() throws SchemaChangeException {
         }
       }
 
-        int probeIndex = -1;
+      int probeIndex = -1;
       // Check if we need to drain the next row in the probe side
       if (getNextRecord) {
-
         if ( !buildSideIsEmpty ) {
           int hashCode = ( cycleNum == 0 ) ?
             partitions[0].getProbeHashCode(recordsProcessed)
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/JoinUtils.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/JoinUtils.java
index e4dab91778..b9745371ee 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/JoinUtils.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/JoinUtils.java
@@ -17,9 +17,22 @@
  */
 package org.apache.drill.exec.physical.impl.join;
 
+import org.apache.calcite.rel.RelShuttleImpl;
 import org.apache.calcite.rel.core.Join;
 import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.TableFunctionScan;
+import org.apache.calcite.rel.logical.LogicalCorrelate;
+import org.apache.calcite.rel.logical.LogicalExchange;
+import org.apache.calcite.rel.logical.LogicalIntersect;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.calcite.rel.logical.LogicalMinus;
+import org.apache.calcite.rel.logical.LogicalSort;
+import org.apache.calcite.rel.logical.LogicalUnion;
+import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.util.Util;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.logical.data.JoinCondition;
 import org.apache.calcite.rel.RelNode;
@@ -35,9 +48,11 @@
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
 import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.planner.logical.DrillLimitRel;
 import org.apache.drill.exec.record.VectorAccessible;
 import org.apache.drill.exec.resolver.TypeCastRules;
 
+import java.util.ArrayList;
 import java.util.LinkedList;
 import java.util.List;
 
@@ -220,7 +235,13 @@ public static boolean isScalarSubquery(RelNode root) {
       if (currentrel instanceof DrillAggregateRel) {
         agg = (DrillAggregateRel)currentrel;
       } else if (currentrel instanceof RelSubset) {
-        currentrel = ((RelSubset)currentrel).getBest() ;
+        currentrel = ((RelSubset) currentrel).getBest() ;
+      } else if (currentrel instanceof DrillLimitRel) {
+        // TODO: Improve this check when DRILL-5691 is fixed.
+        // The problem is that RelMdMaxRowCount currently cannot be used
+        // due to CALCITE-1048.
+        Integer fetchValue = ((RexLiteral) ((DrillLimitRel) currentrel).getFetch()).getValueAs(Integer.class);
+        return fetchValue != null && fetchValue <= 1;
       } else if (currentrel.getInputs().size() == 1) {
         // If the rel is not an aggregate or RelSubset, but is a single-input rel (could be Project,
         // Filter, Sort etc.), check its input
@@ -234,6 +255,17 @@ public static boolean isScalarSubquery(RelNode root) {
       if (agg.getGroupSet().isEmpty()) {
         return true;
       }
+      // Checks that expression in group by is a single and it is literal.
+      // When Calcite rewrites EXISTS sub-queries using SubQueryRemoveRule rules,
+      // it creates project with TRUE literal in expressions list and aggregate on top of it
+      // with empty call list and literal from project expression in group set.
+      if (agg.getAggCallList().isEmpty() && agg.getGroupSet().cardinality() == 1) {
+        ProjectExpressionsCollector expressionsCollector = new ProjectExpressionsCollector();
+        agg.accept(expressionsCollector);
+        List<RexNode> projectedExpressions = expressionsCollector.getProjectedExpressions();
+        return projectedExpressions.size() == 1
+            && RexUtil.isLiteral(projectedExpressions.get(agg.getGroupSet().nth(0)), true);
+      }
     }
     return false;
   }
@@ -267,4 +299,75 @@ public static boolean hasScalarSubqueryInput(RelNode left, RelNode right) {
     return isScalarSubquery(left) || isScalarSubquery(right);
   }
 
+  /**
+   * Collects expressions list from the input project.
+   * For the case when input rel node has single input, its input is taken.
+   */
+  private static class ProjectExpressionsCollector extends RelShuttleImpl {
+    private final List<RexNode> expressions = new ArrayList<>();
+
+    @Override
+    public RelNode visit(RelNode other) {
+      // RelShuttleImpl doesn't have visit methods for Project and RelSubset.
+      if (other instanceof RelSubset) {
+        return visit((RelSubset) other);
+      } else if (other instanceof Project) {
+        return visit((Project) other);
+      }
+      return super.visit(other);
+    }
+
+    @Override
+    public RelNode visit(TableFunctionScan scan) {
+      return scan;
+    }
+
+    @Override
+    public RelNode visit(LogicalJoin join) {
+      return join;
+    }
+
+    @Override
+    public RelNode visit(LogicalCorrelate correlate) {
+      return correlate;
+    }
+
+    @Override
+    public RelNode visit(LogicalUnion union) {
+      return union;
+    }
+
+    @Override
+    public RelNode visit(LogicalIntersect intersect) {
+      return intersect;
+    }
+
+    @Override
+    public RelNode visit(LogicalMinus minus) {
+      return minus;
+    }
+
+    @Override
+    public RelNode visit(LogicalSort sort) {
+      return sort;
+    }
+
+    @Override
+    public RelNode visit(LogicalExchange exchange) {
+      return exchange;
+    }
+
+    private RelNode visit(Project project) {
+      expressions.addAll(project.getProjects());
+      return project;
+    }
+
+    private RelNode visit(RelSubset subset) {
+      return Util.first(subset.getBest(), subset.getOriginal()).accept(this);
+    }
+
+    public List<RexNode> getProjectedExpressions() {
+      return expressions;
+    }
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/LateralJoinBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/LateralJoinBatch.java
index a09913fd8d..fc3c8b1888 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/LateralJoinBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/LateralJoinBatch.java
@@ -20,6 +20,7 @@
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.ExecConstants;
@@ -27,6 +28,7 @@
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.base.LateralContract;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.config.LateralJoinPOP;
 import org.apache.drill.exec.record.AbstractBinaryRecordBatch;
 import org.apache.drill.exec.record.BatchSchema;
@@ -34,10 +36,14 @@
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.RecordBatchSizer;
+import org.apache.drill.exec.record.SchemaBuilder;
 import org.apache.drill.exec.record.VectorAccessibleUtilities;
 import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.vector.ValueVector;
 
+import java.util.HashSet;
+import java.util.List;
+
 import static org.apache.drill.exec.record.RecordBatch.IterOutcome.EMIT;
 import static org.apache.drill.exec.record.RecordBatch.IterOutcome.NONE;
 import static org.apache.drill.exec.record.RecordBatch.IterOutcome.OK;
@@ -52,11 +58,6 @@
 public class LateralJoinBatch extends AbstractBinaryRecordBatch<LateralJoinPOP> implements LateralContract {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(LateralJoinBatch.class);
 
-  // Input indexes to correctly update the stats
-  private static final int LEFT_INPUT = 0;
-
-  private static final int RIGHT_INPUT = 1;
-
   // Maximum number records in the outgoing batch
   private int maxOutputRowCount;
 
@@ -81,8 +82,14 @@
   // Keep track if any matching right record was found for current left index record
   private boolean matchedRecordFound;
 
+  // Used only for testing
   private boolean useMemoryManager = true;
 
+  // Flag to keep track of new left batch so that update on memory manager is called only once per left batch
+  private boolean isNewLeftBatch = false;
+
+  private final HashSet<String> excludedFieldNames = new HashSet<>();
+
   /* ****************************************************************************************************************
    * Public Methods
    * ****************************************************************************************************************/
@@ -92,7 +99,9 @@ public LateralJoinBatch(LateralJoinPOP popConfig, FragmentContext context,
     Preconditions.checkNotNull(left);
     Preconditions.checkNotNull(right);
     final int configOutputBatchSize = (int) context.getOptions().getOption(ExecConstants.OUTPUT_BATCH_SIZE_VALIDATOR);
-    batchMemoryManager = new JoinBatchMemoryManager(configOutputBatchSize, left, right);
+    // Prepare Schema Path Mapping
+    populateExcludedField(popConfig);
+    batchMemoryManager = new JoinBatchMemoryManager(configOutputBatchSize, left, right, excludedFieldNames);
 
     // Initially it's set to default value of 64K and later for each new output row it will be set to the computed
     // row count
@@ -147,9 +156,16 @@ public IterOutcome innerNext() {
     // Setup the references of left, right and outgoing container in generated operator
     state = BatchState.NOT_FIRST;
 
-    // Update the memory manager
-    updateMemoryManager(LEFT_INPUT);
-    updateMemoryManager(RIGHT_INPUT);
+    // Update the memory manager only if its a brand new incoming i.e. leftJoinIndex and rightJoinIndex is 0
+    // Otherwise there will be a case where while filling last output batch, some records from previous left or
+    // right batch are still left to be sent in output for which we will count this batch twice. The actual checks
+    // are done in updateMemoryManager
+    updateMemoryManager(LEFT_INDEX);
+
+    // We have to call update on memory manager for empty batches (rightJoinIndex = -1) as well since other wise while
+    // allocating memory for vectors below it can fail. Since in that case colSize will not have any info on right side
+    // vectors and throws NPE. The actual checks are done in updateMemoryManager
+    updateMemoryManager(RIGHT_INDEX);
 
     // allocate space for the outgoing batch
     allocateVectors();
@@ -161,21 +177,25 @@ public IterOutcome innerNext() {
   public void close() {
     updateBatchMemoryManagerStats();
 
-    logger.debug("BATCH_STATS, incoming aggregate left: batch count : {}, avg bytes : {},  avg row bytes : {}, record count : {}",
-      batchMemoryManager.getNumIncomingBatches(JoinBatchMemoryManager.LEFT_INDEX),
-      batchMemoryManager.getAvgInputBatchSize(JoinBatchMemoryManager.LEFT_INDEX),
-      batchMemoryManager.getAvgInputRowWidth(JoinBatchMemoryManager.LEFT_INDEX),
-      batchMemoryManager.getTotalInputRecords(JoinBatchMemoryManager.LEFT_INDEX));
-
-    logger.debug("BATCH_STATS, incoming aggregate right: batch count : {}, avg bytes : {},  avg row bytes : {}, record count : {}",
-      batchMemoryManager.getNumIncomingBatches(JoinBatchMemoryManager.RIGHT_INDEX),
-      batchMemoryManager.getAvgInputBatchSize(JoinBatchMemoryManager.RIGHT_INDEX),
-      batchMemoryManager.getAvgInputRowWidth(JoinBatchMemoryManager.RIGHT_INDEX),
-      batchMemoryManager.getTotalInputRecords(JoinBatchMemoryManager.RIGHT_INDEX));
-
-    logger.debug("BATCH_STATS, outgoing aggregate: batch count : {}, avg bytes : {},  avg row bytes : {}, record count : {}",
-      batchMemoryManager.getNumOutgoingBatches(), batchMemoryManager.getAvgOutputBatchSize(),
-      batchMemoryManager.getAvgOutputRowWidth(), batchMemoryManager.getTotalOutputRecords());
+    if (logger.isDebugEnabled()) {
+      logger.debug("BATCH_STATS, incoming aggregate left: batch count : {}, avg bytes : {},  avg row bytes : {}, " +
+        "record count : {}", batchMemoryManager.getNumIncomingBatches(JoinBatchMemoryManager.LEFT_INDEX),
+        batchMemoryManager.getAvgInputBatchSize(JoinBatchMemoryManager.LEFT_INDEX),
+        batchMemoryManager.getAvgInputRowWidth(JoinBatchMemoryManager.LEFT_INDEX),
+        batchMemoryManager.getTotalInputRecords(JoinBatchMemoryManager.LEFT_INDEX));
+
+      logger.debug("BATCH_STATS, incoming aggregate right: batch count : {}, avg bytes : {},  avg row bytes : {}, " +
+        "record count : {}", batchMemoryManager.getNumIncomingBatches(JoinBatchMemoryManager.RIGHT_INDEX),
+        batchMemoryManager.getAvgInputBatchSize(JoinBatchMemoryManager.RIGHT_INDEX),
+        batchMemoryManager.getAvgInputRowWidth(JoinBatchMemoryManager.RIGHT_INDEX),
+        batchMemoryManager.getTotalInputRecords(JoinBatchMemoryManager.RIGHT_INDEX));
+
+      logger.debug("BATCH_STATS, outgoing aggregate: batch count : {}, avg bytes : {},  avg row bytes : {}, " +
+        "record count : {}", batchMemoryManager.getNumOutgoingBatches(),
+        batchMemoryManager.getAvgOutputBatchSize(),
+        batchMemoryManager.getAvgOutputRowWidth(),
+        batchMemoryManager.getTotalOutputRecords());
+    }
 
     super.close();
   }
@@ -238,6 +258,7 @@ protected boolean prefetchFirstBatchFromBothSides() {
     boolean validBatch = setBatchState(leftUpstream);
 
     if (validBatch) {
+      isNewLeftBatch = true;
       rightUpstream = next(1, right);
       validBatch = setBatchState(rightUpstream);
     }
@@ -266,10 +287,6 @@ protected void buildSchema() throws SchemaChangeException {
     }
     Preconditions.checkState(right.getRecordCount() == 0, "Unexpected non-empty first right batch received");
 
-    // Update the record memory manager
-    updateMemoryManager(LEFT_INPUT);
-    updateMemoryManager(RIGHT_INPUT);
-
     // Setup output container schema based on known left and right schema
     setupNewSchema();
 
@@ -337,7 +354,12 @@ private IterOutcome processLeftBatch() {
 
     // If left batch is empty
     while (needLeftBatch) {
-      leftUpstream = !processLeftBatchInFuture ? next(LEFT_INPUT, left) : leftUpstream;
+
+      if (!processLeftBatchInFuture) {
+        leftUpstream = next(LEFT_INDEX, left);
+        isNewLeftBatch = true;
+      }
+
       final boolean emptyLeftBatch = left.getRecordCount() <=0;
       logger.trace("Received a left batch and isEmpty: {}", emptyLeftBatch);
 
@@ -418,9 +440,17 @@ private IterOutcome processRightBatch() {
     // will be a valid index. When all records are consumed it will be set to -1.
     boolean needNewRightBatch = (leftJoinIndex >= 0) && (rightJoinIndex == -1);
     while (needNewRightBatch) {
-      rightUpstream = next(RIGHT_INPUT, right);
+      rightUpstream = next(RIGHT_INDEX, right);
       switch (rightUpstream) {
         case OK_NEW_SCHEMA:
+
+          // If there is some records in the output batch that means left batch didn't came with OK_NEW_SCHEMA,
+          // otherwise it would have been marked for processInFuture and output will be returned. This means for
+          // current non processed left or new left non-empty batch there is unexpected right batch schema change
+          if (outputIndex > 0) {
+            throw new IllegalStateException("SchemaChange on right batch is not expected in between the rows of " +
+              "current left batch or a new non-empty left batch with no schema change");
+          }
           // We should not get OK_NEW_SCHEMA multiple times for the same left incoming batch. So there won't be a
           // case where we get OK_NEW_SCHEMA --> OK (with batch) ---> OK_NEW_SCHEMA --> OK/EMIT fall through
           //
@@ -503,7 +533,8 @@ private IterOutcome produceOutputBatch() {
         if (rightUpstream == EMIT) {
           if (!matchedRecordFound && JoinRelType.LEFT == popConfig.getJoinType()) {
             // copy left side in case of LEFT join
-            emitLeft(leftJoinIndex, outputIndex++);
+            emitLeft(leftJoinIndex, outputIndex, 1);
+            ++outputIndex;
           }
           ++leftJoinIndex;
           // Reset matchedRecord for next left index record
@@ -535,6 +566,7 @@ private IterOutcome produceOutputBatch() {
             // Get both left batch and the right batch and make sure indexes are properly set
             leftUpstream = processLeftBatch();
 
+            // output batch is not empty and we have new left batch with OK_NEW_SCHEMA or terminal outcome
             if (processLeftBatchInFuture) {
               logger.debug("Received left batch with outcome {} such that we have to return the current outgoing " +
                 "batch and process the new batch in subsequent next call", leftUpstream);
@@ -551,13 +583,13 @@ private IterOutcome produceOutputBatch() {
 
             // If we have received the left batch with EMIT outcome and is empty then we should return previous output
             // batch with EMIT outcome
-            if (leftUpstream == EMIT && left.getRecordCount() == 0) {
+            if ((leftUpstream == EMIT || leftUpstream == OK_NEW_SCHEMA) && left.getRecordCount() == 0) {
               isLeftProcessed = true;
               break;
             }
 
             // Update the batch memory manager to use new left incoming batch
-            updateMemoryManager(LEFT_INPUT);
+            updateMemoryManager(LEFT_INDEX);
           }
         }
 
@@ -566,10 +598,16 @@ private IterOutcome produceOutputBatch() {
         // left in outgoing batch so let's get next right batch.
         // 2) OR previous left & right batch was fully processed and it came with OK outcome. There is space in outgoing
         // batch. Now we have got new left batch with OK outcome. Let's get next right batch
-        //
-        // It will not hit OK_NEW_SCHEMA since left side have not seen that outcome
+        // 3) OR previous left & right batch was fully processed and left came with OK outcome. Outgoing batch is
+        // empty since all right batches were empty for all left rows. Now we got another non-empty left batch with
+        // OK_NEW_SCHEMA.
         rightUpstream = processRightBatch();
-        Preconditions.checkState(rightUpstream != OK_NEW_SCHEMA, "Unexpected schema change in right branch");
+        if (rightUpstream == OK_NEW_SCHEMA) {
+          leftUpstream = (leftUpstream != EMIT) ? OK : leftUpstream;
+          rightUpstream = OK;
+          finalizeOutputContainer();
+          return OK_NEW_SCHEMA;
+        }
 
         if (isTerminalOutcome(rightUpstream)) {
           finalizeOutputContainer();
@@ -577,7 +615,18 @@ private IterOutcome produceOutputBatch() {
         }
 
         // Update the batch memory manager to use new right incoming batch
-        updateMemoryManager(RIGHT_INPUT);
+        updateMemoryManager(RIGHT_INDEX);
+
+        // If OK_NEW_SCHEMA is seen only on non empty left batch but not on right batch, then we should setup schema in
+        // output container based on new left schema and old right schema. If schema change failed then return STOP
+        // downstream
+        if (leftUpstream == OK_NEW_SCHEMA && isLeftProcessed) {
+          if (!handleSchemaChange()) {
+            return STOP;
+          }
+          // Since schema has change so we have new empty vectors in output container hence allocateMemory for them
+          allocateVectors();
+        }
       }
     } // output batch is full to its max capacity
 
@@ -615,9 +664,11 @@ private void finalizeOutputContainer() {
 
     batchMemoryManager.updateOutgoingStats(outputIndex);
 
-    logger.debug("BATCH_STATS, outgoing:\n {}", new RecordBatchSizer(this));
-    logger.debug("Number of records emitted: {} and Allocator Stats: [AllocatedMem: {}, PeakMem: {}]", outputIndex,
-      container.getAllocator().getAllocatedMemory(), container.getAllocator().getPeakMemoryAllocation());
+    if (logger.isDebugEnabled()) {
+      logger.debug("BATCH_STATS, outgoing:\n {}", new RecordBatchSizer(this));
+      logger.debug("Number of records emitted: {} and Allocator Stats: [AllocatedMem: {}, PeakMem: {}]",
+        outputIndex, container.getAllocator().getAllocatedMemory(), container.getAllocator().getPeakMemoryAllocation());
+    }
 
     // Update the output index for next output batch to zero
     outputIndex = 0;
@@ -659,6 +710,21 @@ private boolean verifyInputSchema(BatchSchema schema) {
     return isValid;
   }
 
+  private BatchSchema batchSchemaWithNoExcludedCols(BatchSchema originSchema) {
+    if (excludedFieldNames.size() == 0) {
+      return originSchema;
+    }
+
+    final SchemaBuilder newSchemaBuilder =
+      BatchSchema.newBuilder().setSelectionVectorMode(originSchema.getSelectionVectorMode());
+    for (MaterializedField field : originSchema) {
+      if (!excludedFieldNames.contains(field.getName())) {
+        newSchemaBuilder.addField(field);
+      }
+    }
+    return newSchemaBuilder.build();
+  }
+
   /**
    * Helps to create the outgoing container vectors based on known left and right batch schemas
    * @throws SchemaChangeException
@@ -670,8 +736,8 @@ private void setupNewSchema() throws SchemaChangeException {
 
     // Clear up the container
     container.clear();
-    leftSchema = left.getSchema();
-    rightSchema = right.getSchema();
+    leftSchema = batchSchemaWithNoExcludedCols(left.getSchema());
+    rightSchema = batchSchemaWithNoExcludedCols(right.getSchema());
 
     if (!verifyInputSchema(leftSchema)) {
       throw new SchemaChangeException("Invalid Schema found for left incoming batch");
@@ -683,12 +749,20 @@ private void setupNewSchema() throws SchemaChangeException {
 
     // Setup LeftSchema in outgoing container
     for (final VectorWrapper<?> vectorWrapper : left) {
-      container.addOrGet(vectorWrapper.getField());
+      final MaterializedField leftField = vectorWrapper.getField();
+      if (excludedFieldNames.contains(leftField.getName())) {
+        continue;
+      }
+      container.addOrGet(leftField);
     }
 
     // Setup RightSchema in the outgoing container
     for (final VectorWrapper<?> vectorWrapper : right) {
       MaterializedField rightField = vectorWrapper.getField();
+      if (excludedFieldNames.contains(rightField.getName())) {
+        continue;
+      }
+
       TypeProtos.MajorType rightFieldType = vectorWrapper.getField().getType();
 
       // make right input schema optional if we have LEFT join
@@ -720,6 +794,9 @@ private void allocateVectors() {
       RecordBatchSizer.ColumnSize colSize = batchMemoryManager.getColumnSize(w.getField().getName());
       colSize.allocateVector(w.getValueVector(), maxOutputRowCount);
     }
+
+    logger.debug("Allocator Stats: [AllocatedMem: {}, PeakMem: {}]", container.getAllocator().getAllocatedMemory(),
+      container.getAllocator().getPeakMemoryAllocation());
   }
 
   private boolean setBatchState(IterOutcome outcome) {
@@ -745,8 +822,6 @@ private boolean setBatchState(IterOutcome outcome) {
    * this left index. The right container is copied starting from rightIndex until number of records in the container.
    */
   private void crossJoinAndOutputRecords() {
-    logger.trace("Producing output for leftIndex: {}, rightIndex: {}, rightRecordCount: {} and outputIndex: {}",
-      leftJoinIndex, rightJoinIndex, right.getRecordCount(), outputIndex);
     final int rightRecordCount = right.getRecordCount();
 
     // If there is no record in right batch just return current index in output batch
@@ -756,16 +831,30 @@ private void crossJoinAndOutputRecords() {
 
     // Check if right batch is empty since we have to handle left join case
     Preconditions.checkState(rightJoinIndex != -1, "Right batch record count is >0 but index is -1");
-    // For every record in right side just emit left and right records in output container
-    for (int i = rightJoinIndex; i < rightRecordCount; ++i) {
-      emitLeft(leftJoinIndex, outputIndex);
-      emitRight(i, outputIndex);
-      ++outputIndex;
-
-      if (isOutgoingBatchFull()) {
-        break;
-      }
+
+    int currentOutIndex = outputIndex;
+    // Number of rows that can be copied in output batch
+    final int maxAvailableRowSlot = maxOutputRowCount - currentOutIndex;
+    // Number of rows that can be copied inside output batch is minimum of available slot in
+    // output batch and available data to copy from right side. It can be half consumed right batch
+    // which has few more rows to be copied to output but output batch has more to fill.
+    final int rowsToCopy = Math.min(maxAvailableRowSlot, (rightRecordCount - rightJoinIndex));
+
+    if (logger.isDebugEnabled()) {
+      logger.debug("Producing output for leftIndex: {}, rightIndex: {}, rightRecordCount: {}, outputIndex: {} and " +
+        "availableSlotInOutput: {}", leftJoinIndex, rightJoinIndex, rightRecordCount, outputIndex, maxAvailableRowSlot);
+      logger.debug("Output Batch stats before copying new data: {}", new RecordBatchSizer(this));
     }
+
+    // First copy all the left vectors data. Doing it in this way since it's the same data being copied over may be
+    // we will have performance gain from JVM
+    emitLeft(leftJoinIndex, currentOutIndex, rowsToCopy);
+
+    // Copy all the right side vectors data
+    emitRight(rightJoinIndex, currentOutIndex, rowsToCopy);
+
+    // Update outputIndex
+    outputIndex += rowsToCopy;
   }
 
   /**
@@ -779,27 +868,49 @@ private void crossJoinAndOutputRecords() {
    * @param startVectorIndex - start index of vector inside source record batch
    * @param endVectorIndex - end index of vector inside source record batch
    * @param baseVectorIndex - base index to be added to startVectorIndex to get corresponding vector in outgoing batch
+   * @param numRowsToCopy - Number of rows to copy into output batch
+   * @param moveFromIndex - boolean to indicate if the fromIndex should also be increased or not. Since in case of
+   *                      copying data from left vector fromIndex is constant whereas in case of copying data from right
+   *                      vector fromIndex will move along with output index.
    */
   private void copyDataToOutputVectors(int fromRowIndex, int toRowIndex, RecordBatch batch,
-                                      int startVectorIndex, int endVectorIndex, int baseVectorIndex) {
+                                       int startVectorIndex, int endVectorIndex, int baseVectorIndex,
+                                       int numRowsToCopy, boolean moveFromIndex) {
     // Get the vectors using field index rather than Materialized field since input batch field can be different from
     // output container field in case of Left Join. As we rebuild the right Schema field to be optional for output
     // container.
+    int inputIndex = 0;
     for (int i = startVectorIndex; i < endVectorIndex; ++i) {
-      // Get input vector
-      final Class<?> inputValueClass = batch.getSchema().getColumn(i).getValueClass();
-      final ValueVector inputVector = batch.getValueAccessorById(inputValueClass, i).getValueVector();
-
       // Get output vector
       final int outputVectorIndex = i + baseVectorIndex;
       final Class<?> outputValueClass = this.getSchema().getColumn(outputVectorIndex).getValueClass();
       final ValueVector outputVector = this.getValueAccessorById(outputValueClass, outputVectorIndex).getValueVector();
+      final String outputFieldName = outputVector.getField().getName();
+
+      ValueVector inputVector;
+      Class<?> inputValueClass;
+      String inputFieldName;
+      do {
+        // Get input vector
+        inputValueClass = batch.getSchema().getColumn(inputIndex).getValueClass();
+        inputVector = batch.getValueAccessorById(inputValueClass, inputIndex).getValueVector();
+        inputFieldName = inputVector.getField().getName();
+        ++inputIndex;
+      } while (excludedFieldNames.contains(inputFieldName));
+
+      Preconditions.checkArgument(outputFieldName.equals(inputFieldName),
+        new IllegalStateException(String.format("Non-excluded Input and output container fields are not in same order" +
+          ". Output Schema:%s and Input Schema:%s", this.getSchema(), batch.getSchema())));
 
       logger.trace("Copying data from incoming batch vector to outgoing batch vector. [IncomingBatch: " +
-          "(RowIndex: {}, VectorType: {}), OutputBatch: (RowIndex: {}, VectorType: {}) and BaseIndex: {}]",
-        fromRowIndex, inputValueClass, toRowIndex, outputValueClass, baseVectorIndex);
-      // Copy data from input vector to output vector
-      outputVector.copyEntry(toRowIndex, inputVector, fromRowIndex);
+          "(RowIndex: {}, VectorType: {}), OutputBatch: (RowIndex: {}, VectorType: {}) and Other: (TimeEachValue: {}," +
+          " NumBaseIndex: {}) ]",
+        fromRowIndex, inputValueClass, toRowIndex, outputValueClass, numRowsToCopy, baseVectorIndex);
+
+      // Copy data from input vector to output vector for numRowsToCopy times.
+      for (int j = 0; j < numRowsToCopy; ++j) {
+        outputVector.copyEntry(toRowIndex + j, inputVector, (moveFromIndex) ? fromRowIndex + j : fromRowIndex);
+      }
     }
   }
 
@@ -809,8 +920,9 @@ private void copyDataToOutputVectors(int fromRowIndex, int toRowIndex, RecordBat
    * @param leftIndex - index to copy data from left incoming batch vectors
    * @param outIndex - index to copy data to in outgoing batch vectors
    */
-  private void emitLeft(int leftIndex, int outIndex) {
-    copyDataToOutputVectors(leftIndex, outIndex, left, 0, leftSchema.getFieldCount(), 0);
+  private void emitLeft(int leftIndex, int outIndex, int numRowsToCopy) {
+    copyDataToOutputVectors(leftIndex, outIndex, left, 0,
+      leftSchema.getFieldCount(), 0, numRowsToCopy, false);
   }
 
   /**
@@ -819,8 +931,9 @@ private void emitLeft(int leftIndex, int outIndex) {
    * @param rightIndex - index to copy data from right incoming batch vectors
    * @param outIndex - index to copy data to in outgoing batch vectors
    */
-  private void emitRight(int rightIndex, int outIndex) {
-    copyDataToOutputVectors(rightIndex, outIndex, right, 0, rightSchema.getFieldCount(), leftSchema.getFieldCount());
+  private void emitRight(int rightIndex, int outIndex, int numRowsToCopy) {
+    copyDataToOutputVectors(rightIndex, outIndex, right, 0,
+      rightSchema.getFieldCount(), leftSchema.getFieldCount(), numRowsToCopy, true);
   }
 
   /**
@@ -847,16 +960,37 @@ private boolean isOutgoingBatchFull() {
   }
 
   private void updateMemoryManager(int inputIndex) {
+
+    if (inputIndex == LEFT_INDEX && isNewLeftBatch) {
+      // reset state and continue to update
+      isNewLeftBatch = false;
+    } else if (inputIndex == RIGHT_INDEX && (rightJoinIndex == 0 || rightJoinIndex == -1)) {
+      // continue to update
+    } else {
+      return;
+    }
+
     // For cases where all the previous input were consumed and send with previous output batch. But now we are building
     // a new output batch with new incoming then it will not cause any problem since outputIndex will be 0
     final int newOutputRowCount = batchMemoryManager.update(inputIndex, outputIndex);
 
     if (logger.isDebugEnabled()) {
-      logger.debug("BATCH_STATS, incoming {}:\n {}", inputIndex == 0 ? "left" : "right", batchMemoryManager.getRecordBatchSizer(inputIndex));
+      logger.debug("BATCH_STATS, incoming {}:\n {}", inputIndex == LEFT_INDEX ? "left" : "right",
+        batchMemoryManager.getRecordBatchSizer(inputIndex));
+      logger.debug("Previous OutputRowCount: {}, New OutputRowCount: {}", maxOutputRowCount, newOutputRowCount);
     }
 
     if (useMemoryManager) {
       maxOutputRowCount = newOutputRowCount;
     }
   }
+
+  private void populateExcludedField(PhysicalOperator lateralPop) {
+    final List<SchemaPath> excludedCols = ((LateralJoinPOP)lateralPop).getExcludedColumns();
+    if (excludedCols != null) {
+      for (SchemaPath currentPath : excludedCols) {
+        excludedFieldNames.add(currentPath.rootName());
+      }
+    }
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatch.java
index a5c2ae7318..ea34ed930e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatch.java
@@ -62,6 +62,7 @@
 import org.apache.drill.exec.vector.complex.AbstractContainerVector;
 
 import java.io.IOException;
+import java.util.HashSet;
 import java.util.List;
 
 import static org.apache.drill.exec.compile.sig.GeneratorMapping.GM;
@@ -108,7 +109,7 @@
   private class MergeJoinMemoryManager extends JoinBatchMemoryManager {
 
     MergeJoinMemoryManager(int outputBatchSize, RecordBatch leftBatch, RecordBatch rightBatch) {
-      super(outputBatchSize, leftBatch, rightBatch);
+      super(outputBatchSize, leftBatch, rightBatch, new HashSet<>());
     }
 
     /**
@@ -122,9 +123,7 @@
     @Override
     public void update(int inputIndex) {
       status.setTargetOutputRowCount(super.update(inputIndex, status.getOutPosition()));
-      if (logger.isDebugEnabled()) {
-        logger.debug("BATCH_STATS, incoming {}:\n {}", inputIndex == 0 ? "left" : "right", getRecordBatchSizer(inputIndex));
-      }
+      logger.debug("BATCH_STATS, incoming {}: {}", inputIndex == 0 ? "left" : "right", getRecordBatchSizer(inputIndex));
     }
   }
 
@@ -132,8 +131,10 @@ protected MergeJoinBatch(MergeJoinPOP popConfig, FragmentContext context, Record
     super(popConfig, context, true, left, right);
 
     // Instantiate the batch memory manager
-    final int outputBatchSize = (int) context.getOptions().getOption(ExecConstants.OUTPUT_BATCH_SIZE_VALIDATOR);
-    batchMemoryManager = new MergeJoinMemoryManager(outputBatchSize, left, right);
+    final int configuredBatchSize = (int) context.getOptions().getOption(ExecConstants.OUTPUT_BATCH_SIZE_VALIDATOR);
+    batchMemoryManager = new MergeJoinMemoryManager(configuredBatchSize, left, right);
+
+    logger.debug("BATCH_STATS, configured output batch size: {}", configuredBatchSize);
 
     if (popConfig.getConditions().size() == 0) {
       throw new UnsupportedOperationException("Merge Join currently does not support cartesian join.  This join operator was configured with 0 conditions");
@@ -271,7 +272,7 @@ private void setRecordCountInContainer() {
     }
 
     if (logger.isDebugEnabled()) {
-      logger.debug("BATCH_STATS, outgoing:\n {}", new RecordBatchSizer(this));
+      logger.debug("BATCH_STATS, outgoing: {}", new RecordBatchSizer(this));
     }
 
     batchMemoryManager.updateOutgoingStats(getRecordCount());
@@ -281,21 +282,23 @@ private void setRecordCountInContainer() {
   public void close() {
     updateBatchMemoryManagerStats();
 
-    logger.debug("BATCH_STATS, incoming aggregate left: batch count : {}, avg bytes : {},  avg row bytes : {}, record count : {}",
-      batchMemoryManager.getNumIncomingBatches(JoinBatchMemoryManager.LEFT_INDEX),
-      batchMemoryManager.getAvgInputBatchSize(JoinBatchMemoryManager.LEFT_INDEX),
-      batchMemoryManager.getAvgInputRowWidth(JoinBatchMemoryManager.LEFT_INDEX),
-      batchMemoryManager.getTotalInputRecords(JoinBatchMemoryManager.LEFT_INDEX));
-
-    logger.debug("BATCH_STATS, incoming aggregate right: batch count : {}, avg bytes : {},  avg row bytes : {}, record count : {}",
-      batchMemoryManager.getNumIncomingBatches(JoinBatchMemoryManager.RIGHT_INDEX),
-      batchMemoryManager.getAvgInputBatchSize(JoinBatchMemoryManager.RIGHT_INDEX),
-      batchMemoryManager.getAvgInputRowWidth(JoinBatchMemoryManager.RIGHT_INDEX),
-      batchMemoryManager.getTotalInputRecords(JoinBatchMemoryManager.RIGHT_INDEX));
-
-    logger.debug("BATCH_STATS, outgoing aggregate: batch count : {}, avg bytes : {},  avg row bytes : {}, record count : {}",
-      batchMemoryManager.getNumOutgoingBatches(), batchMemoryManager.getAvgOutputBatchSize(),
-      batchMemoryManager.getAvgOutputRowWidth(), batchMemoryManager.getTotalOutputRecords());
+    if (logger.isDebugEnabled()) {
+      logger.debug("BATCH_STATS, incoming aggregate left: batch count : {}, avg bytes : {},  avg row bytes : {}, record count : {}",
+        batchMemoryManager.getNumIncomingBatches(JoinBatchMemoryManager.LEFT_INDEX),
+        batchMemoryManager.getAvgInputBatchSize(JoinBatchMemoryManager.LEFT_INDEX),
+        batchMemoryManager.getAvgInputRowWidth(JoinBatchMemoryManager.LEFT_INDEX),
+        batchMemoryManager.getTotalInputRecords(JoinBatchMemoryManager.LEFT_INDEX));
+
+      logger.debug("BATCH_STATS, incoming aggregate right: batch count : {}, avg bytes : {},  avg row bytes : {}, record count : {}",
+        batchMemoryManager.getNumIncomingBatches(JoinBatchMemoryManager.RIGHT_INDEX),
+        batchMemoryManager.getAvgInputBatchSize(JoinBatchMemoryManager.RIGHT_INDEX),
+        batchMemoryManager.getAvgInputRowWidth(JoinBatchMemoryManager.RIGHT_INDEX),
+        batchMemoryManager.getTotalInputRecords(JoinBatchMemoryManager.RIGHT_INDEX));
+
+      logger.debug("BATCH_STATS, outgoing aggregate: batch count : {}, avg bytes : {},  avg row bytes : {}, record count : {}",
+        batchMemoryManager.getNumOutgoingBatches(), batchMemoryManager.getAvgOutputBatchSize(),
+        batchMemoryManager.getAvgOutputRowWidth(), batchMemoryManager.getTotalOutputRecords());
+    }
 
     super.close();
     leftIterator.close();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/NestedLoopJoin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/NestedLoopJoin.java
index f7d96ad8c2..725c46d1ba 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/NestedLoopJoin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/NestedLoopJoin.java
@@ -37,6 +37,9 @@ public void setupNestedLoopJoin(FragmentContext context, RecordBatch left,
                                   ExpandableHyperContainer rightContainer,
                                   LinkedList<Integer> rightCounts,
                                   NestedLoopJoinBatch outgoing);
+
+  void setTargetOutputCount(int targetOutputCount);
+
   // Produce output records taking into account join type
   public int outputRecords(JoinRelType joinType);
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/NestedLoopJoinBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/NestedLoopJoinBatch.java
index ae14fb3ec9..e2532e8ed2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/NestedLoopJoinBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/NestedLoopJoinBatch.java
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.physical.impl.join;
 
 import java.io.IOException;
+import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.Map;
 
@@ -29,6 +30,7 @@
 import org.apache.drill.common.expression.LogicalExpression;
 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.compile.sig.GeneratorMapping;
 import org.apache.drill.exec.compile.sig.MappingSet;
 import org.apache.drill.exec.exception.ClassTransformationException;
@@ -50,8 +52,8 @@
 import org.apache.drill.exec.record.TypedFieldId;
 import org.apache.drill.exec.record.VectorAccessible;
 import org.apache.drill.exec.record.VectorWrapper;
-import org.apache.drill.exec.vector.AllocationHelper;
-
+import org.apache.drill.exec.record.JoinBatchMemoryManager;
+import org.apache.drill.exec.record.RecordBatchSizer;
 import com.google.common.base.Preconditions;
 import com.sun.codemodel.JExpr;
 import com.sun.codemodel.JExpression;
@@ -65,9 +67,6 @@
 public class NestedLoopJoinBatch extends AbstractBinaryRecordBatch<NestedLoopJoinPOP> {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(NestedLoopJoinBatch.class);
 
-  // Maximum number records in the outgoing batch
-  protected static final int MAX_BATCH_SIZE = 4096;
-
   // Input indexes to correctly update the stats
   protected static final int LEFT_INPUT = 0;
   protected static final int RIGHT_INPUT = 1;
@@ -130,6 +129,11 @@ protected NestedLoopJoinBatch(NestedLoopJoinPOP popConfig, FragmentContext conte
     super(popConfig, context, left, right);
     Preconditions.checkNotNull(left);
     Preconditions.checkNotNull(right);
+
+    // get the output batch size from config.
+    int configuredBatchSize = (int) context.getOptions().getOption(ExecConstants.OUTPUT_BATCH_SIZE_VALIDATOR);
+    batchMemoryManager = new JoinBatchMemoryManager(configuredBatchSize, left, right, new HashSet<>());
+    logger.debug("BATCH_STATS, configured output batch size: {}", configuredBatchSize);
   }
 
   /**
@@ -162,6 +166,9 @@ public IterOutcome innerNext() {
             }
             // fall through
           case OK:
+            // For right side, use aggregate i.e. average row width across batches
+            batchMemoryManager.update(RIGHT_INDEX, 0, true);
+            logger.debug("BATCH_STATS, incoming right: {}", batchMemoryManager.getRecordBatchSizer(RIGHT_INDEX));
             addBatchToHyperContainer(right);
             break;
           case OUT_OF_MEMORY:
@@ -179,7 +186,9 @@ public IterOutcome innerNext() {
     }
 
     // allocate space for the outgoing batch
-    allocateVectors();
+    batchMemoryManager.allocateVectors(container);
+
+    nljWorker.setTargetOutputCount(batchMemoryManager.getOutputRowCount());
 
     // invoke the runtime generated method to emit records in the output batch
     outputRecords = nljWorker.outputRecords(popConfig.getJoinType());
@@ -193,6 +202,10 @@ public IterOutcome innerNext() {
     container.setRecordCount(outputRecords);
     container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
 
+    if (logger.isDebugEnabled()) {
+      logger.debug("BATCH_STATS, outgoing: {}", new RecordBatchSizer(this));
+    }
+
     logger.debug("Number of records emitted: " + outputRecords);
 
     return (outputRecords > 0) ? IterOutcome.OK : IterOutcome.NONE;
@@ -331,15 +344,6 @@ private NestedLoopJoin setupWorker() throws IOException, ClassTransformationExce
     return context.getImplementationClass(nLJCodeGenerator);
   }
 
-  /**
-   * Simple method to allocate space for all the vectors in the container.
-   */
-  private void allocateVectors() {
-    for (final VectorWrapper<?> vw : container) {
-      AllocationHelper.allocateNew(vw.getValueVector(), MAX_BATCH_SIZE);
-    }
-  }
-
   /**
    * Builds the output container's schema. Goes over the left and the right
    * batch and adds the corresponding vectors to the output container.
@@ -352,6 +356,9 @@ protected void buildSchema() throws SchemaChangeException {
         return;
       }
 
+      batchMemoryManager.update(RIGHT_INDEX, 0, true);
+      logger.debug("BATCH_STATS, incoming right: {}", batchMemoryManager.getRecordBatchSizer(RIGHT_INDEX));
+
       if (leftUpstream != IterOutcome.NONE) {
         leftSchema = left.getSchema();
         for (final VectorWrapper<?> vw : left) {
@@ -380,7 +387,6 @@ protected void buildSchema() throws SchemaChangeException {
         addBatchToHyperContainer(right);
       }
 
-      allocateVectors();
       nljWorker = setupWorker();
 
       // if left batch is empty, fetch next
@@ -388,7 +394,9 @@ protected void buildSchema() throws SchemaChangeException {
         leftUpstream = next(LEFT_INPUT, left);
       }
 
-      container.setRecordCount(0);
+      batchMemoryManager.update(LEFT_INDEX, 0);
+      logger.debug("BATCH_STATS, incoming left: {}", batchMemoryManager.getRecordBatchSizer(LEFT_INDEX));
+
       container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
 
     } catch (ClassTransformationException | IOException e) {
@@ -412,6 +420,26 @@ private void addBatchToHyperContainer(RecordBatch inputBatch) {
 
   @Override
   public void close() {
+    updateBatchMemoryManagerStats();
+
+    if (logger.isDebugEnabled()) {
+      logger.debug("BATCH_STATS, incoming aggregate left: batch count : {}, avg bytes : {},  avg row bytes : {}, record count : {}",
+              batchMemoryManager.getNumIncomingBatches(JoinBatchMemoryManager.LEFT_INDEX),
+              batchMemoryManager.getAvgInputBatchSize(JoinBatchMemoryManager.LEFT_INDEX),
+              batchMemoryManager.getAvgInputRowWidth(JoinBatchMemoryManager.LEFT_INDEX),
+              batchMemoryManager.getTotalInputRecords(JoinBatchMemoryManager.LEFT_INDEX));
+
+      logger.debug("BATCH_STATS, incoming aggregate right: batch count : {}, avg bytes : {},  avg row bytes : {}, record count : {}",
+              batchMemoryManager.getNumIncomingBatches(JoinBatchMemoryManager.RIGHT_INDEX),
+              batchMemoryManager.getAvgInputBatchSize(JoinBatchMemoryManager.RIGHT_INDEX),
+              batchMemoryManager.getAvgInputRowWidth(JoinBatchMemoryManager.RIGHT_INDEX),
+              batchMemoryManager.getTotalInputRecords(JoinBatchMemoryManager.RIGHT_INDEX));
+
+      logger.debug("BATCH_STATS, outgoing aggregate: batch count : {}, avg bytes : {},  avg row bytes : {}, record count : {}",
+              batchMemoryManager.getNumOutgoingBatches(), batchMemoryManager.getAvgOutputBatchSize(),
+              batchMemoryManager.getAvgOutputRowWidth(), batchMemoryManager.getTotalOutputRecords());
+    }
+
     rightContainer.clear();
     rightCounts.clear();
     super.close();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/NestedLoopJoinTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/NestedLoopJoinTemplate.java
index cdd02f42cb..adf681b58e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/NestedLoopJoinTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/NestedLoopJoinTemplate.java
@@ -29,12 +29,16 @@
 import java.util.LinkedList;
 import java.util.List;
 
+import static org.apache.drill.exec.record.JoinBatchMemoryManager.LEFT_INDEX;
+
 /*
  * Template class that combined with the runtime generated source implements the NestedLoopJoin interface. This
  * class contains the main nested loop join logic.
  */
 public abstract class NestedLoopJoinTemplate implements NestedLoopJoin {
 
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(NestedLoopJoinBatch.class);
+
   // Current left input batch being processed
   private RecordBatch left = null;
 
@@ -50,6 +54,8 @@
   // Iteration status tracker
   private IterationStatusTracker tracker = new IterationStatusTracker();
 
+  private int targetOutputRecords;
+
   /**
    * Method initializes necessary state and invokes the doSetup() to set the
    * input and output value vector references.
@@ -69,10 +75,14 @@ public void setupNestedLoopJoin(FragmentContext context,
     this.leftRecordCount = left.getRecordCount();
     this.rightCounts = rightCounts;
     this.outgoing = outgoing;
-
     doSetup(context, rightContainer, left, outgoing);
   }
 
+  @Override
+  public void setTargetOutputCount(int targetOutputRecords) {
+    this.targetOutputRecords = targetOutputRecords;
+  }
+
   /**
    * Main entry point for producing the output records. Thin wrapper around populateOutgoingBatch(), this method
    * controls which left batch we are processing and fetches the next left input batch once we exhaust the current one.
@@ -84,11 +94,11 @@ public int outputRecords(JoinRelType joinType) {
     int outputIndex = 0;
     while (leftRecordCount != 0) {
       outputIndex = populateOutgoingBatch(joinType, outputIndex);
-      if (outputIndex >= NestedLoopJoinBatch.MAX_BATCH_SIZE) {
+      if (outputIndex >= targetOutputRecords) {
         break;
       }
       // reset state and get next left batch
-      resetAndGetNextLeft();
+      resetAndGetNextLeft(outputIndex);
     }
     return outputIndex;
   }
@@ -128,7 +138,7 @@ private int populateOutgoingBatch(JoinRelType joinType, int outputIndex) {
             outputIndex++;
             rightRecordMatched = true;
 
-            if (outputIndex >= NestedLoopJoinBatch.MAX_BATCH_SIZE) {
+            if (outputIndex >= targetOutputRecords) {
               nextRightRecordToProcess++;
 
               // no more space left in the batch, stop processing
@@ -143,7 +153,7 @@ private int populateOutgoingBatch(JoinRelType joinType, int outputIndex) {
         // project records from the left side only, records from right will be null
         emitLeft(nextLeftRecordToProcess, outputIndex);
         outputIndex++;
-        if (outputIndex >= NestedLoopJoinBatch.MAX_BATCH_SIZE) {
+        if (outputIndex >= targetOutputRecords) {
           nextLeftRecordToProcess++;
 
           // no more space left in the batch, stop processing
@@ -165,7 +175,7 @@ private int populateOutgoingBatch(JoinRelType joinType, int outputIndex) {
    * Resets some internal state which indicates the next records to process in the left and right batches,
    * also fetches the next left input batch.
    */
-  private void resetAndGetNextLeft() {
+  private void resetAndGetNextLeft(int outputIndex) {
     for (VectorWrapper<?> vw : left) {
       vw.getValueVector().clear();
     }
@@ -181,6 +191,8 @@ private void resetAndGetNextLeft() {
         leftRecordCount = 0;
         break;
       case OK:
+        setTargetOutputCount(outgoing.getBatchMemoryManager().update(left, LEFT_INDEX,outputIndex));
+        logger.debug("BATCH_STATS, incoming left: {}", outgoing.getBatchMemoryManager().getRecordBatchSizer(LEFT_INDEX));
         leftRecordCount = left.getRecordCount();
         break;
     }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/OutputSizeEstimateConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/OutputSizeEstimateConstants.java
new file mode 100644
index 0000000000..1c65508d8c
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/OutputSizeEstimateConstants.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.project;
+
+public class OutputSizeEstimateConstants {
+    public static final int USER_NAME_LENGTH = 32; //libc useradd limit
+    public static final int SCHEMA_LENGTH = 1024;
+    public static final int USER_ID_LENGTH = 32;   //UUID length
+    public static final int DATE_TIME_LENGTH = 100; //DateTypeFunctions timeofday truncates to 100
+    public static final int CONVERT_TO_FLOAT_LENGTH = 4; //float 4 to varbinary
+    public static final int CONVERT_TO_TINYINT_LENGTH = 1; // tiny int to varbinary
+    public static final int CONVERT_TO_INT_LENGTH = 4; // INT to BigEndian Int
+    public static final int CONVERT_TO_BIGINT_LENGTH = 8; // convert to BigEndianBigInt/BigInt
+    public static final int CONVERT_TO_UINT4_LENGTH = 4; // convert_toUINT4
+    public static final int CONVERT_TO_SMALLINT_LENGTH = 2; // convert_toSMALLINT_BE
+    public static final int CONVERT_TO_TIME_EPOCH_LENGTH = 8; // convert_toTIME_EPOCH_BE
+    public static final int CONVERT_TO_DOUBLE_LENGTH = 8; // convert_to_double_be
+    public static final int CONVERT_TO_BOOLEAN_BYTE_LENGTH = 1; // tiny int to varbinary
+    public static final int CONVERT_TO_DATE_EPOCH_LENGTH = 8; // tiny int to varbinary
+    public static final int CONVERT_TO_TIMESTAMP_EPOCH_LENGTH = 8; // tiny int to varbinary
+    public static final int CONVERT_TO_HADOOPV_LENGTH = 9; // Hadoop Variable length integer. 1 - 9 bytes
+    public static final int CONVERT_TO_UINT8_LENGTH = 8; // uint8 length
+
+    public static final int CHAR_LENGTH = 1;
+
+    //TODO Make this a user config?
+    public static final int NON_DRILL_FUNCTION_OUTPUT_SIZE_ESTIMATE = 50;
+
+    //TODO Make this a user config?
+    public static final int COMPLEX_FIELD_ESTIMATE = 50;
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/OutputWidthExpression.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/OutputWidthExpression.java
new file mode 100644
index 0000000000..b9240d68be
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/OutputWidthExpression.java
@@ -0,0 +1,147 @@
+/*
+ * 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.project;
+
+import org.apache.drill.common.expression.FunctionHolderExpression;
+import org.apache.drill.exec.expr.AbstractExecExprVisitor;
+import org.apache.drill.exec.expr.ValueVectorReadExpression;
+import org.apache.drill.exec.expr.fn.output.OutputWidthCalculator;
+
+import java.util.ArrayList;
+
+/**
+ * OutputWidthExpressions are used to capture the information required to calculate the width of the output
+ * produced by a variable-width expression. This is used by the {@link ProjectMemoryManager} to calculate output-widths of the expressions
+ * being projected. Expressions in Drill are represented as a tree of {@link org.apache.drill.common.expression.LogicalExpression}.
+ * During the setup phase, the {@link OutputWidthVisitor} walks the tree of LogicalExpressions and reduces it to a tree of
+ * OutputWidthExpressions. In the execution phase, the OutputWidthVisitor walks the tree of OutputWidthExpressions and
+ * reduces it to a fixed output-width by using the average-sizes of incoming columns obtained from the
+ * {@link org.apache.drill.exec.record.RecordBatchSizer}
+ *
+ */
+public abstract class OutputWidthExpression {
+
+    abstract <T, V, E extends Exception> T accept(AbstractExecExprVisitor<T, V, E> visitor, V value) throws E;
+
+    /**
+     * IfElseWidthExpr is uded to capture an {@link org.apache.drill.common.expression.IfExpression}. The max of the if-side width and
+     * else-side width will be used as the expression width.
+     */
+    public static class IfElseWidthExpr extends OutputWidthExpression {
+        OutputWidthExpression[] expressions;
+
+        public IfElseWidthExpr(OutputWidthExpression ifExpr, OutputWidthExpression elseExpr) {
+            this.expressions = new OutputWidthExpression[2];
+            this.expressions[0] = ifExpr;
+            this.expressions[1] = elseExpr;
+        }
+
+        @Override
+        public <T, V, E extends Exception> T accept(AbstractExecExprVisitor<T, V, E> visitor, V value) throws E {
+            return visitor.visitIfElseWidthExpr(this, value);
+        }
+
+    }
+
+    /**
+     * FunctionCallExpr captures the details required to calculate the width of the output produced by a function
+     * that produces variable-width output. It captures the {@link OutputWidthCalculator} for the function and the
+     * arguments.
+     */
+    public static class FunctionCallExpr extends OutputWidthExpression {
+        FunctionHolderExpression holder;
+        ArrayList<OutputWidthExpression> args;
+        OutputWidthCalculator widthCalculator;
+
+        public FunctionCallExpr(FunctionHolderExpression holder, OutputWidthCalculator widthCalculator,
+                                ArrayList<OutputWidthExpression> args) {
+            this.holder = holder;
+            this.args = args;
+            this.widthCalculator = widthCalculator;
+        }
+
+        public FunctionHolderExpression getHolder() {
+            return holder;
+        }
+
+        public ArrayList<OutputWidthExpression> getArgs() {
+            return args;
+        }
+
+        public OutputWidthCalculator getCalculator() {
+            return widthCalculator;
+        }
+
+        @Override
+        public <T, V, E extends Exception> T accept(AbstractExecExprVisitor<T, V, E> visitor, V value) throws E {
+            return visitor.visitFunctionCallExpr(this, value);
+        }
+    }
+
+    /**
+     * VarLenReadExpr captures the name of a variable length column that is used (read) in an expression.
+     * The captured name will be used to lookup the average entry size for the column in the corresponding
+     * {@link org.apache.drill.exec.record.RecordBatchSizer}
+     */
+    public static class VarLenReadExpr extends OutputWidthExpression  {
+        ValueVectorReadExpression readExpression;
+        String name;
+
+        public VarLenReadExpr(ValueVectorReadExpression readExpression) {
+            this.readExpression = readExpression;
+            this.name = null;
+        }
+
+        public VarLenReadExpr(String name) {
+            this.readExpression = null;
+            this.name = name;
+        }
+
+        public ValueVectorReadExpression getReadExpression() {
+            return readExpression;
+        }
+
+        public String getName() {
+            return name;
+        }
+
+        @Override
+        public <T, V, E extends Exception> T accept(AbstractExecExprVisitor<T, V, E> visitor, V value) throws E {
+            return visitor.visitVarLenReadExpr(this, value);
+        }
+    }
+
+    /**
+     * Used to represent fixed-width values used in an expression.
+     */
+
+    public static class FixedLenExpr extends OutputWidthExpression {
+        int fixedWidth;
+        public FixedLenExpr(int fixedWidth) {
+            this.fixedWidth = fixedWidth;
+        }
+        public int getWidth() { return fixedWidth;}
+
+        @Override
+        public <T, V, E extends Exception> T accept(AbstractExecExprVisitor<T, V, E> visitor, V value) throws E {
+            return visitor.visitFixedLenExpr(this, value);
+        }
+    }
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/OutputWidthVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/OutputWidthVisitor.java
new file mode 100644
index 0000000000..cb58795230
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/OutputWidthVisitor.java
@@ -0,0 +1,278 @@
+/*
+ * 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.project;
+
+import com.google.common.base.Preconditions;
+import org.apache.drill.common.expression.FunctionHolderExpression;
+import org.apache.drill.common.expression.IfExpression;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.TypedNullConstant;
+import org.apache.drill.common.expression.ValueExpressions;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.expr.AbstractExecExprVisitor;
+import org.apache.drill.exec.expr.DrillFuncHolderExpr;
+import org.apache.drill.exec.expr.ValueVectorReadExpression;
+import org.apache.drill.exec.expr.ValueVectorWriteExpression;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.fn.DrillFuncHolder;
+import org.apache.drill.exec.expr.fn.output.OutputWidthCalculator;
+import org.apache.drill.exec.physical.impl.project.OutputWidthExpression.FixedLenExpr;
+import org.apache.drill.exec.physical.impl.project.OutputWidthExpression.FunctionCallExpr;
+import org.apache.drill.exec.physical.impl.project.OutputWidthExpression.VarLenReadExpr;
+import org.apache.drill.exec.physical.impl.project.OutputWidthExpression.IfElseWidthExpr;
+import org.apache.drill.common.expression.ValueExpressions.VarDecimalExpression;
+import org.apache.drill.exec.record.RecordBatchSizer;
+import org.apache.drill.exec.record.TypedFieldId;
+
+import java.util.ArrayList;
+
+public class OutputWidthVisitor extends AbstractExecExprVisitor<OutputWidthExpression, OutputWidthVisitorState,
+        RuntimeException> {
+
+    @Override
+    public OutputWidthExpression visitVarDecimalConstant(VarDecimalExpression varDecimalExpression,
+                                                         OutputWidthVisitorState state) throws RuntimeException {
+        Preconditions.checkArgument(varDecimalExpression.getMajorType().hasPrecision());
+        return new FixedLenExpr(varDecimalExpression.getMajorType().getPrecision());
+    }
+
+
+    /**
+     *
+     * Records the {@link IfExpression} as a {@link IfElseWidthExpr}. IfElseWidthExpr will be reduced to
+     * a {@link FixedLenExpr} by taking the max of the if-expr-width and the else-expr-width.
+     *
+     * @param ifExpression
+     * @param state
+     * @return IfElseWidthExpr
+     * @throws RuntimeException
+     */
+    @Override
+    public OutputWidthExpression visitIfExpression(IfExpression ifExpression, OutputWidthVisitorState state)
+                                                                    throws RuntimeException {
+        IfExpression.IfCondition condition = ifExpression.ifCondition;
+        LogicalExpression ifExpr = condition.expression;
+        LogicalExpression elseExpr = ifExpression.elseExpression;
+
+        OutputWidthExpression ifWidthExpr = ifExpr.accept(this, state);
+        OutputWidthExpression elseWidthExpr = null;
+        if (elseExpr != null) {
+            elseWidthExpr = elseExpr.accept(this, state);
+        }
+        return new IfElseWidthExpr(ifWidthExpr, elseWidthExpr);
+    }
+
+    /**
+     * Handles a {@link FunctionHolderExpression}. Functions that produce fixed-width output are trivially
+     * converted to a {@link FixedLenExpr}. For functions that produce variable width output, the output width calculator
+     * annotation is looked-up and recorded in a {@link FunctionCallExpr}. This calculator will later be used to convert
+     * the FunctionCallExpr to a {@link FixedLenExpr} expression
+     * @param holderExpr
+     * @param state
+     * @return FunctionCallExpr
+     * @throws RuntimeException
+     */
+    @Override
+    public OutputWidthExpression visitFunctionHolderExpression(FunctionHolderExpression holderExpr,
+                                                               OutputWidthVisitorState state) throws RuntimeException {
+        OutputWidthExpression fixedWidth = getFixedLenExpr(holderExpr.getMajorType());
+        if (fixedWidth != null) { return fixedWidth; }
+        // Only Drill functions can be handled. Non-drill Functions, like HiveFunctions
+        // will default to a fixed value
+        if (!(holderExpr instanceof DrillFuncHolderExpr)) {
+            // We currently only know how to handle DrillFuncs.
+            // Use a default if this is not a DrillFunc
+            return new FixedLenExpr(OutputSizeEstimateConstants.NON_DRILL_FUNCTION_OUTPUT_SIZE_ESTIMATE);
+        }
+
+        final DrillFuncHolder holder = ((DrillFuncHolderExpr) holderExpr).getHolder();
+
+        // If the user has provided a size estimate, use it
+        int estimate = holder.variableOutputSizeEstimate();
+        if (estimate != FunctionTemplate.OUTPUT_SIZE_ESTIMATE_DEFAULT) {
+            return new FixedLenExpr(estimate);
+        }
+        // Use the calculator provided by the user or use the default
+        OutputWidthCalculator widthCalculator = holder.getOutputWidthCalculator();
+        final int argSize = holderExpr.args.size();
+        ArrayList<OutputWidthExpression> arguments = null;
+        if (argSize != 0) {
+            arguments = new ArrayList<>(argSize);
+            for (LogicalExpression expr : holderExpr.args) {
+                arguments.add(expr.accept(this, state));
+            }
+        }
+        return new FunctionCallExpr(holderExpr, widthCalculator, arguments);
+    }
+
+    /**
+     * Records a variable width write expression. This will be converted to a {@link FixedLenExpr} expression by walking
+     * the tree of expression attached to the write expression.
+     * @param writeExpr
+     * @param state
+     * @return
+     * @throws RuntimeException
+     */
+    @Override
+    public OutputWidthExpression visitValueVectorWriteExpression(ValueVectorWriteExpression writeExpr,
+                                                                 OutputWidthVisitorState state) throws RuntimeException {
+        TypedFieldId fieldId = writeExpr.getFieldId();
+        ProjectMemoryManager manager = state.getManager();
+        OutputWidthExpression outputExpr;
+        if (manager.isFixedWidth(fieldId)) {
+            outputExpr = getFixedLenExpr(fieldId.getFinalType());
+        } else {
+            LogicalExpression writeArg = writeExpr.getChild();
+            outputExpr = writeArg.accept(this, state);
+        }
+        return outputExpr;
+    }
+
+    /**
+     * Records a variable width read expression as a {@link VarLenReadExpr}. This will be converted to a
+     * {@link FixedLenExpr} expression by getting the size for the corresponding column from the {@link RecordBatchSizer}.
+     *
+     * @param readExpr
+     * @param state
+     * @return
+     * @throws RuntimeException
+     */
+    @Override
+    public OutputWidthExpression visitValueVectorReadExpression(ValueVectorReadExpression readExpr,
+                                                                OutputWidthVisitorState state) throws RuntimeException {
+        return new VarLenReadExpr(readExpr);
+    }
+
+    @Override
+    public OutputWidthExpression visitQuotedStringConstant(ValueExpressions.QuotedString quotedString,
+                                                           OutputWidthVisitorState state) throws RuntimeException {
+        return new FixedLenExpr(quotedString.getString().length());
+    }
+
+    @Override
+    public OutputWidthExpression visitUnknown(LogicalExpression logicalExpression, OutputWidthVisitorState state) {
+        OutputWidthExpression fixedLenExpr = getFixedLenExpr(logicalExpression.getMajorType());
+        if (fixedLenExpr != null) {
+            return fixedLenExpr;
+        }
+        throw new IllegalStateException("Unknown variable width expression: " + logicalExpression);
+    }
+
+    @Override
+    public OutputWidthExpression visitNullConstant(TypedNullConstant nullConstant, OutputWidthVisitorState state)
+            throws RuntimeException {
+        int width;
+        if (nullConstant.getMajorType().hasPrecision()) {
+            width = nullConstant.getMajorType().getPrecision();
+        } else {
+            width = 0;
+        }
+        return new FixedLenExpr(width);
+    }
+
+
+    @Override
+    public OutputWidthExpression visitFixedLenExpr(FixedLenExpr fixedLenExpr, OutputWidthVisitorState state)
+            throws RuntimeException {
+        return fixedLenExpr;
+    }
+
+    /**
+     * Converts the {@link VarLenReadExpr} to a {@link FixedLenExpr} by getting the size for the corresponding column
+     * from the RecordBatchSizer.
+     * @param varLenReadExpr
+     * @param state
+     * @return
+     * @throws RuntimeException
+     */
+    @Override
+    public OutputWidthExpression visitVarLenReadExpr(VarLenReadExpr varLenReadExpr, OutputWidthVisitorState state)
+                                                        throws RuntimeException {
+        String columnName = varLenReadExpr.getName();
+        if (columnName == null) {
+            TypedFieldId fieldId = varLenReadExpr.getReadExpression().getTypedFieldId();
+            columnName =  TypedFieldId.getPath(fieldId, state.manager.getIncomingBatch());
+        }
+        final RecordBatchSizer.ColumnSize columnSize = state.manager.getColumnSize(columnName);
+
+        int columnWidth = columnSize.getNetSizePerEntry();
+        return new FixedLenExpr(columnWidth);
+    }
+
+    /**
+     * Converts a {@link FunctionCallExpr} to a {@link FixedLenExpr} by passing the the args of the function to the
+     * width calculator for this function.
+     * @param functionCallExpr
+     * @param state
+     * @return
+     * @throws RuntimeException
+     */
+    @Override
+    public OutputWidthExpression visitFunctionCallExpr(FunctionCallExpr functionCallExpr, OutputWidthVisitorState state)
+                                                        throws RuntimeException {
+        ArrayList<OutputWidthExpression> args = functionCallExpr.getArgs();
+        ArrayList<FixedLenExpr> estimatedArgs = null;
+
+        if (args != null && args.size() != 0) {
+            estimatedArgs = new ArrayList<>(args.size());
+            for (OutputWidthExpression expr : args) {
+                // Once the args are visited, they will all become FixedWidthExpr
+                FixedLenExpr fixedLenExpr = (FixedLenExpr) expr.accept(this, state);
+                estimatedArgs.add(fixedLenExpr);
+            }
+        }
+        OutputWidthCalculator estimator = functionCallExpr.getCalculator();
+        int estimatedSize = estimator.getOutputWidth(estimatedArgs);
+        return new FixedLenExpr(estimatedSize);
+    }
+
+    /**
+     *  Converts the {@link IfElseWidthExpr}  to a {@link FixedLenExpr} by taking the max of the if-expr-width and the
+     *  else-expr-width.
+     * @param ifElseWidthExpr
+     * @param state
+     * @return
+     * @throws RuntimeException
+     */
+    @Override
+    public OutputWidthExpression visitIfElseWidthExpr(IfElseWidthExpr ifElseWidthExpr, OutputWidthVisitorState state)
+                                                        throws RuntimeException {
+        OutputWidthExpression ifReducedExpr = ifElseWidthExpr.expressions[0].accept(this, state);
+        assert ifReducedExpr instanceof FixedLenExpr;
+        int ifWidth = ((FixedLenExpr)ifReducedExpr).getWidth();
+        int elseWidth = -1;
+        if (ifElseWidthExpr.expressions[1] != null) {
+            OutputWidthExpression elseReducedExpr = ifElseWidthExpr.expressions[1].accept(this, state);
+            assert elseReducedExpr instanceof FixedLenExpr;
+            elseWidth = ((FixedLenExpr)elseReducedExpr).getWidth();
+        }
+        int outputWidth = Math.max(ifWidth, elseWidth);
+        return new FixedLenExpr(outputWidth);
+    }
+
+    private OutputWidthExpression getFixedLenExpr(MajorType majorType) {
+        MajorType type = majorType;
+        if (Types.isFixedWidthType(type)) {
+            int fixedWidth = ProjectMemoryManager.getWidthOfFixedWidthType(type);
+            return new OutputWidthExpression.FixedLenExpr(fixedWidth);
+        }
+        return null;
+    }
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/OutputWidthVisitorState.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/OutputWidthVisitorState.java
new file mode 100644
index 0000000000..c0e0cb1c9e
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/OutputWidthVisitorState.java
@@ -0,0 +1,37 @@
+/*
+ * 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.project;
+
+public class OutputWidthVisitorState {
+
+    ProjectMemoryManager manager;
+    ProjectMemoryManager.OutputColumnType outputColumnType;
+
+    public OutputWidthVisitorState(ProjectMemoryManager manager, ProjectMemoryManager.OutputColumnType outputColumnType) {
+        this.manager = manager;
+        this.outputColumnType = outputColumnType;
+    }
+
+    public ProjectMemoryManager getManager() {
+        return manager;
+    }
+
+    public ProjectMemoryManager.OutputColumnType getOutputColumnType() {
+        return outputColumnType;
+    }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectMemoryManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectMemoryManager.java
new file mode 100644
index 0000000000..f461b09228
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectMemoryManager.java
@@ -0,0 +1,310 @@
+/*
+ * 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.project;
+
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.project.OutputWidthExpression.VarLenReadExpr;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.RecordBatchMemoryManager;
+import org.apache.drill.exec.record.RecordBatchSizer;
+import org.apache.drill.exec.record.TypedFieldId;
+import org.apache.drill.exec.vector.FixedWidthVector;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.physical.impl.project.OutputWidthExpression.FixedLenExpr;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ *
+ * ProjectMemoryManager(PMM) is used to estimate the size of rows produced by ProjectRecordBatch.
+ * The PMM works as follows:
+ *
+ * Setup phase: As and when ProjectRecordBatch creates or transfers a field, it registers the field with PMM.
+ * If the field is a variable width field, PMM records the expression that produces the variable
+ * width field. The expression is a tree of LogicalExpressions. The PMM walks this tree of LogicalExpressions
+ * to produce a tree of OutputWidthExpressions. The widths of Fixed width fields are just accumulated into a single
+ * total. Note: The PMM, currently, cannot handle new complex fields, it just uses a hard-coded estimate for such fields.
+ *
+ *
+ * Execution phase: Just before a batch is processed by Project, the PMM walks the tree of OutputWidthExpressions
+ * and converts them to FixedWidthExpressions. It uses the RecordBatchSizer and the function annotations to do this conversion.
+ * See OutputWidthVisitor for details.
+ */
+public class ProjectMemoryManager extends RecordBatchMemoryManager {
+
+    static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ProjectMemoryManager.class);
+
+    public RecordBatch getIncomingBatch() {
+        return incomingBatch;
+    }
+
+    RecordBatch incomingBatch = null;
+    ProjectRecordBatch outgoingBatch = null;
+
+    int rowWidth = 0;
+    Map<String, ColumnWidthInfo> outputColumnSizes;
+    // Number of variable width columns in the batch
+    int variableWidthColumnCount = 0;
+    // Number of fixed width columns in the batch
+    int fixedWidthColumnCount = 0;
+    // Number of complex columns in the batch
+    int complexColumnsCount = 0;
+
+
+    // Holds sum of all fixed width column widths
+    int totalFixedWidthColumnWidth = 0;
+    // Holds sum of all complex column widths
+    // Currently, this is just a guess
+    int totalComplexColumnWidth = 0;
+
+    enum WidthType {
+        FIXED,
+        VARIABLE
+    }
+
+    enum OutputColumnType {
+        TRANSFER,
+        NEW
+    }
+
+    class ColumnWidthInfo {
+        //MaterializedField materializedField;
+        OutputWidthExpression outputExpression;
+        int width;
+        WidthType widthType;
+        OutputColumnType outputColumnType;
+        String name;
+
+        ColumnWidthInfo(ValueVector vv,
+                        OutputWidthExpression outputWidthExpression,
+                        OutputColumnType outputColumnType,
+                        WidthType widthType,
+                        int fieldWidth) {
+            this.outputExpression = outputWidthExpression;
+            this.width = fieldWidth;
+            this.outputColumnType = outputColumnType;
+            this.widthType = widthType;
+            String columnName = vv.getField().getName();
+            this.name = columnName;
+        }
+
+        public OutputWidthExpression getOutputExpression() { return outputExpression; }
+
+        public OutputColumnType getOutputColumnType() { return outputColumnType; }
+
+        boolean isFixedWidth() { return widthType == WidthType.FIXED; }
+
+        public int getWidth() { return width; }
+
+        public String getName() { return name; }
+    }
+
+    void ShouldNotReachHere() {
+        throw new IllegalStateException();
+    }
+
+    private void setIncomingBatch(RecordBatch recordBatch) {
+        incomingBatch = recordBatch;
+    }
+
+    private void setOutgoingBatch(ProjectRecordBatch outgoingBatch) {
+        this.outgoingBatch = outgoingBatch;
+    }
+
+    public ProjectMemoryManager(int configuredOutputSize) {
+        super(configuredOutputSize);
+        outputColumnSizes = new HashMap<>();
+        logger.debug("BATCH_STATS, configuredOutputSize: {}", configuredOutputSize);
+    }
+
+    public boolean isComplex(MajorType majorType) {
+        MinorType minorType = majorType.getMinorType();
+        return minorType == MinorType.MAP || minorType == MinorType.UNION || minorType == MinorType.LIST;
+    }
+
+    boolean isFixedWidth(TypedFieldId fieldId) {
+        ValueVector vv = getOutgoingValueVector(fieldId);
+        return isFixedWidth(vv);
+    }
+
+    public ValueVector getOutgoingValueVector(TypedFieldId fieldId) {
+        Class<?> clazz = fieldId.getIntermediateClass();
+        int[] fieldIds = fieldId.getFieldIds();
+        return outgoingBatch.getValueAccessorById(clazz, fieldIds).getValueVector();
+    }
+
+    static boolean isFixedWidth(ValueVector vv) {  return (vv instanceof FixedWidthVector); }
+
+
+    static int getWidthOfFixedWidthType(ValueVector vv) {
+        assert isFixedWidth(vv);
+        return ((FixedWidthVector)vv).getValueWidth();
+    }
+
+    public static int getWidthOfFixedWidthType(TypeProtos.MajorType majorType) {
+        DataMode mode = majorType.getMode();
+        MinorType minorType = majorType.getMinorType();
+        final boolean isVariableWidth  = (minorType == MinorType.VARCHAR || minorType == MinorType.VAR16CHAR
+                || minorType == MinorType.VARBINARY);
+
+        if (isVariableWidth) {
+            throw new IllegalArgumentException("getWidthOfFixedWidthType() cannot handle variable width types");
+        }
+
+        final boolean isOptional = (mode == DataMode.OPTIONAL);
+        final boolean isRepeated = (mode == DataMode.REPEATED);
+        final boolean isRepeatedList = false; // repeated
+        final Map<String, RecordBatchSizer.ColumnSize> children = null;
+
+        return RecordBatchSizer.getStdNetSizePerEntryCommon(majorType, isOptional, isRepeated, isRepeatedList, children);
+    }
+
+
+    void addTransferField(ValueVector vvOut, String path) {
+        addField(vvOut, null, OutputColumnType.TRANSFER, path);
+    }
+
+    void addNewField(ValueVector vv, LogicalExpression logicalExpression) {
+        addField(vv, logicalExpression, OutputColumnType.NEW, null);
+    }
+
+    void addField(ValueVector vv, LogicalExpression logicalExpression, OutputColumnType outputColumnType, String path) {
+        if(isFixedWidth(vv)) {
+            addFixedWidthField(vv);
+        } else {
+            addVariableWidthField(vv, logicalExpression, outputColumnType, path);
+        }
+    }
+
+    private void addVariableWidthField(ValueVector vv, LogicalExpression logicalExpression,
+                                       OutputColumnType outputColumnType, String path) {
+        variableWidthColumnCount++;
+        ColumnWidthInfo columnWidthInfo;
+        //Variable width transfers
+        if(outputColumnType == OutputColumnType.TRANSFER) {
+            String columnName = path;
+            VarLenReadExpr readExpr = new VarLenReadExpr(columnName);
+            columnWidthInfo = new ColumnWidthInfo(vv, readExpr, outputColumnType,
+                    WidthType.VARIABLE, -1); //fieldWidth has to be obtained from the RecordBatchSizer
+        } else if (isComplex(vv.getField().getType())) {
+            addComplexField(vv);
+            return;
+        } else {
+            // Walk the tree of LogicalExpressions to get a tree of OutputWidthExpressions
+            OutputWidthVisitorState state = new OutputWidthVisitorState(this, outputColumnType);
+            OutputWidthExpression outputWidthExpression = logicalExpression.accept(new OutputWidthVisitor(), state);
+            columnWidthInfo = new ColumnWidthInfo(vv, outputWidthExpression, outputColumnType,
+                    WidthType.VARIABLE, -1); //fieldWidth has to be obtained from the OutputWidthExpression
+        }
+        outputColumnSizes.put(columnWidthInfo.getName(), columnWidthInfo);
+    }
+
+    void addComplexField(ValueVector vv) {
+        //Complex types are not yet supported. Just use a guess for the size
+        assert vv == null || isComplex(vv.getField().getType());
+        complexColumnsCount++;
+        // just a guess
+        totalComplexColumnWidth +=  OutputSizeEstimateConstants.COMPLEX_FIELD_ESTIMATE;
+    }
+
+    void addFixedWidthField(ValueVector vv) {
+        assert isFixedWidth(vv);
+        fixedWidthColumnCount++;
+        int fixedFieldWidth = getWidthOfFixedWidthType(vv);
+        totalFixedWidthColumnWidth += fixedFieldWidth;
+    }
+
+    public void init(RecordBatch incomingBatch, ProjectRecordBatch outgoingBatch) {
+        setIncomingBatch(incomingBatch);
+        setOutgoingBatch(outgoingBatch);
+        reset();
+    }
+
+    private void reset() {
+        rowWidth = 0;
+        totalFixedWidthColumnWidth = 0;
+        totalComplexColumnWidth = 0;
+
+        fixedWidthColumnCount = 0;
+        complexColumnsCount = 0;
+    }
+
+    @Override
+    public void update() {
+        long updateStartTime = System.currentTimeMillis();
+        RecordBatchSizer batchSizer = new RecordBatchSizer(incomingBatch);
+        long batchSizerEndTime = System.currentTimeMillis();
+
+        setRecordBatchSizer(batchSizer);
+        rowWidth = 0;
+        int totalVariableColumnWidth = 0;
+        for (String expr : outputColumnSizes.keySet()) {
+            ColumnWidthInfo columnWidthInfo = outputColumnSizes.get(expr);
+            int width = -1;
+            if (columnWidthInfo.isFixedWidth()) {
+                // fixed width columns are accumulated in totalFixedWidthColumnWidth
+                ShouldNotReachHere();
+            } else {
+                //Walk the tree of OutputWidthExpressions to get a FixedLenExpr
+                //As the tree is walked, the RecordBatchSizer and function annotations
+                //are looked-up to come up with the final FixedLenExpr
+                OutputWidthExpression savedWidthExpr = columnWidthInfo.getOutputExpression();
+                OutputColumnType columnType = columnWidthInfo.getOutputColumnType();
+                OutputWidthVisitorState state = new OutputWidthVisitorState(this, columnType);
+                OutputWidthExpression reducedExpr = savedWidthExpr.accept(new OutputWidthVisitor(), state);
+                assert reducedExpr instanceof FixedLenExpr;
+                width = ((FixedLenExpr)reducedExpr).getWidth();
+                assert width >= 0;
+            }
+            totalVariableColumnWidth += width;
+        }
+        rowWidth += totalFixedWidthColumnWidth;
+        rowWidth += totalComplexColumnWidth;
+        rowWidth += totalVariableColumnWidth;
+        int outPutRowCount;
+        if (rowWidth != 0) {
+            //if rowWidth is not zero, set the output row count in the sizer
+            setOutputRowCount(getOutputBatchSize(), rowWidth);
+            // if more rows can be allowed than the incoming row count, then set the
+            // output row count to the incoming row count.
+            outPutRowCount = Math.min(getOutputRowCount(), batchSizer.rowCount());
+        } else {
+            // if rowWidth == 0 then the memory manager does
+            // not have sufficient information to size the batch
+            // let the entire batch pass through.
+            // If incoming rc == 0, all RB Sizer look-ups will have
+            // 0 width and so total width can be 0
+            outPutRowCount = incomingBatch.getRecordCount();
+        }
+        setOutputRowCount(outPutRowCount);
+        long updateEndTime = System.currentTimeMillis();
+        logger.trace("update() : Output RC {}, BatchSizer RC {}, incoming RC {}, width {}, total fixed width {}"
+                    + ", total variable width {}, total complex width {}, batchSizer time {} ms, update time {}  ms"
+                    + ", manager {}, incoming {}",outPutRowCount, batchSizer.rowCount(), incomingBatch.getRecordCount(),
+                    totalFixedWidthColumnWidth, totalVariableColumnWidth, totalComplexColumnWidth,
+                    (batchSizerEndTime - updateStartTime),(updateEndTime - updateStartTime), this, incomingBatch);
+
+        logger.debug("BATCH_STATS, incoming: {}", getRecordBatchSizer());
+        updateIncomingStats();
+    }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
index 8a88db9fef..4bc63c0b1b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
@@ -37,6 +37,7 @@
 import org.apache.drill.common.logical.data.NamedExpression;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.exception.ClassTransformationException;
 import org.apache.drill.exec.exception.OutOfMemoryException;
 import org.apache.drill.exec.exception.SchemaChangeException;
@@ -54,6 +55,7 @@
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.RecordBatchSizer;
 import org.apache.drill.exec.record.SimpleRecordBatch;
 import org.apache.drill.exec.record.TransferPair;
 import org.apache.drill.exec.record.TypedFieldId;
@@ -83,6 +85,9 @@
   private int remainderIndex = 0;
   private int recordCount;
 
+  private ProjectMemoryManager memoryManager;
+
+
   private static final String EMPTY_STRING = "";
   private boolean first = true;
   private boolean wasNone = false; // whether a NONE iter outcome was already seen
@@ -108,6 +113,11 @@ private void clear() {
 
   public ProjectRecordBatch(final Project pop, final RecordBatch incoming, final FragmentContext context) throws OutOfMemoryException {
     super(pop, context, incoming);
+
+    // get the output batch size from config.
+    int configuredBatchSize = (int) context.getOptions().getOption(ExecConstants.OUTPUT_BATCH_SIZE_VALIDATOR);
+
+    memoryManager = new ProjectMemoryManager(configuredBatchSize);
   }
 
   @Override
@@ -150,14 +160,18 @@ protected IterOutcome doWork() {
 
     int incomingRecordCount = incoming.getRecordCount();
 
+    logger.trace("doWork(): incoming rc {}, incoming {}, Project {}", incomingRecordCount, incoming, this);
+    //calculate the output row count
+    memoryManager.update();
+
     if (first && incomingRecordCount == 0) {
       if (complexWriters != null) {
         IterOutcome next = null;
         while (incomingRecordCount == 0) {
           if (getLastKnownOutcome() == EMIT) {
             throw new UnsupportedOperationException("Currently functions producing complex types as output is not " +
-              "supported in project list for subquery between LATERAL and UNNEST. Please re-write the query using this " +
-              "function in the projection list of outermost query.");
+                    "supported in project list for subquery between LATERAL and UNNEST. Please re-write the query using this " +
+                    "function in the projection list of outermost query.");
           }
 
           next = next(incoming);
@@ -177,7 +191,7 @@ protected IterOutcome doWork() {
             // been setup during setupNewSchema
             for (FieldReference fieldReference : complexFieldReferencesList) {
               MaterializedField field = MaterializedField.create(fieldReference.getAsNamePart().getName(),
-                UntypedNullHolder.TYPE);
+                      UntypedNullHolder.TYPE);
               container.add(new UntypedNullVector(field, container.getAllocator()));
             }
             container.buildSchema(SelectionVectorMode.NONE);
@@ -193,6 +207,9 @@ protected IterOutcome doWork() {
             }
           }
           incomingRecordCount = incoming.getRecordCount();
+          memoryManager.update();
+          logger.trace("doWork():[1] memMgr RC {}, incoming rc {},  incoming {}, Project {}",
+                       memoryManager.getOutputRowCount(), incomingRecordCount, incoming, this);
         }
       }
     }
@@ -206,12 +223,21 @@ protected IterOutcome doWork() {
     first = false;
     container.zeroVectors();
 
-    if (!doAlloc(incomingRecordCount)) {
+
+    int maxOuputRecordCount = memoryManager.getOutputRowCount();
+    logger.trace("doWork():[2] memMgr RC {}, incoming rc {}, incoming {}, project {}",
+                 memoryManager.getOutputRowCount(), incomingRecordCount, incoming, this);
+
+    if (!doAlloc(maxOuputRecordCount)) {
       outOfMemory = true;
       return IterOutcome.OUT_OF_MEMORY;
     }
+    long projectStartTime = System.currentTimeMillis();
+    final int outputRecords = projector.projectRecords(this.incoming,0, maxOuputRecordCount, 0);
+    long projectEndTime = System.currentTimeMillis();
+    logger.trace("doWork(): projection: records {}, time {} ms", outputRecords, (projectEndTime - projectStartTime));
+
 
-    final int outputRecords = projector.projectRecords(0, incomingRecordCount, 0);
     if (outputRecords < incomingRecordCount) {
       setValueCount(outputRecords);
       hasRemainder = true;
@@ -230,6 +256,9 @@ protected IterOutcome doWork() {
       container.buildSchema(SelectionVectorMode.NONE);
     }
 
+    memoryManager.updateOutgoingStats(outputRecords);
+    logger.debug("BATCH_STATS, outgoing: {}", new RecordBatchSizer(this));
+
     // Get the final outcome based on hasRemainder since that will determine if all the incoming records were
     // consumed in current output batch or not
     return getFinalOutcome(hasRemainder);
@@ -237,11 +266,23 @@ protected IterOutcome doWork() {
 
   private void handleRemainder() {
     final int remainingRecordCount = incoming.getRecordCount() - remainderIndex;
-    if (!doAlloc(remainingRecordCount)) {
+    assert this.memoryManager.incomingBatch == incoming;
+    final int recordsToProcess = Math.min(remainingRecordCount, memoryManager.getOutputRowCount());
+
+    if (!doAlloc(recordsToProcess)) {
       outOfMemory = true;
       return;
     }
-    final int projRecords = projector.projectRecords(remainderIndex, remainingRecordCount, 0);
+
+    logger.trace("handleRemainder: remaining RC {}, toProcess {}, remainder index {}, incoming {}, Project {}",
+                 remainingRecordCount, recordsToProcess, remainderIndex, incoming, this);
+
+    long projectStartTime = System.currentTimeMillis();
+    final int projRecords = projector.projectRecords(this.incoming, remainderIndex, recordsToProcess, 0);
+    long projectEndTime = System.currentTimeMillis();
+
+    logger.trace("handleRemainder: projection: " + "records {}, time {} ms", projRecords,(projectEndTime - projectStartTime));
+
     if (projRecords < remainingRecordCount) {
       setValueCount(projRecords);
       this.recordCount = projRecords;
@@ -260,6 +301,9 @@ private void handleRemainder() {
     if (complexWriters != null) {
       container.buildSchema(SelectionVectorMode.NONE);
     }
+
+    memoryManager.updateOutgoingStats(projRecords);
+    logger.debug("BATCH_STATS, outgoing: {}", new RecordBatchSizer(this));
   }
 
   public void addComplexWriter(final ComplexWriter writer) {
@@ -314,7 +358,7 @@ private boolean isAnyWildcard(final List<NamedExpression> exprs) {
   }
 
   private boolean isWildcard(final NamedExpression ex) {
-    if ( !(ex.getExpr() instanceof SchemaPath)) {
+    if (!(ex.getExpr() instanceof SchemaPath)) {
       return false;
     }
     final NameSegment expr = ((SchemaPath)ex.getExpr()).getRootSegment();
@@ -322,6 +366,8 @@ private boolean isWildcard(final NamedExpression ex) {
   }
 
   private void setupNewSchemaFromInput(RecordBatch incomingBatch) throws SchemaChangeException {
+    long setupNewSchemaStartTime = System.currentTimeMillis();
+    memoryManager.init(incomingBatch, this);
     if (allocationVectors != null) {
       for (final ValueVector v : allocationVectors) {
         v.clear();
@@ -332,6 +378,7 @@ private void setupNewSchemaFromInput(RecordBatch incomingBatch) throws SchemaCha
     if (complexWriters != null) {
       container.clear();
     } else {
+      // Release the underlying DrillBufs and reset the ValueVectors to empty
       // Not clearing the container here is fine since Project output schema is not determined solely based on incoming
       // batch. It is defined by the expressions it has to evaluate.
       //
@@ -347,7 +394,7 @@ private void setupNewSchemaFromInput(RecordBatch incomingBatch) throws SchemaCha
     final ClassGenerator<Projector> cg = CodeGenerator.getRoot(Projector.TEMPLATE_DEFINITION, context.getOptions());
     cg.getCodeGenerator().plainJavaCapable(true);
     // Uncomment out this line to debug the generated code.
-    // cg.getCodeGenerator().saveCodeForDebugging(true);
+    //cg.getCodeGenerator().saveCodeForDebugging(true);
 
     final IntHashSet transferFieldIds = new IntHashSet();
 
@@ -358,7 +405,6 @@ private void setupNewSchemaFromInput(RecordBatch incomingBatch) throws SchemaCha
 
     for (NamedExpression namedExpression : exprs) {
       result.clear();
-
       if (classify && namedExpression.getExpr() instanceof SchemaPath) {
         classifyExpr(namedExpression, incomingBatch, result);
 
@@ -385,6 +431,7 @@ private void setupNewSchemaFromInput(RecordBatch incomingBatch) throws SchemaCha
               final ValueVector vvOut = container.addOrGet(MaterializedField.create(ref.getAsNamePart().getName(),
                 vvIn.getField().getType()), callBack);
               final TransferPair tp = vvIn.makeTransferPair(vvOut);
+              memoryManager.addTransferField(vvIn, vvIn.getField().getName());
               transfers.add(tp);
             }
           } else if (value != null && value > 1) { // subsequent wildcards should do a copy of incoming valuevectors
@@ -414,6 +461,7 @@ private void setupNewSchemaFromInput(RecordBatch incomingBatch) throws SchemaCha
               allocationVectors.add(vv);
               final TypedFieldId fid = container.getValueVectorId(SchemaPath.getSimplePath(outputField.getName()));
               final ValueVectorWriteExpression write = new ValueVectorWriteExpression(fid, expr, true);
+              memoryManager.addNewField(vv, write);
               final HoldingContainer hc = cg.addExpr(write, ClassGenerator.BlkCreateMode.TRUE_IF_BOUND);
             }
           }
@@ -423,10 +471,9 @@ private void setupNewSchemaFromInput(RecordBatch incomingBatch) throws SchemaCha
         // For the columns which do not needed to be classified,
         // it is still necessary to ensure the output column name is unique
         result.outputNames = Lists.newArrayList();
-        final String outputName = getRef(namedExpression).getRootSegment().getPath();
+        final String outputName = getRef(namedExpression).getRootSegment().getPath(); //moved to before the if
         addToResultMaps(outputName, result, true);
       }
-
       String outputName = getRef(namedExpression).getRootSegment().getPath();
       if (result != null && result.outputNames != null && result.outputNames.size() > 0) {
         boolean isMatched = false;
@@ -466,6 +513,7 @@ private void setupNewSchemaFromInput(RecordBatch incomingBatch) throws SchemaCha
           container.addOrGet(MaterializedField.create(ref.getLastSegment().getNameSegment().getPath(),
             vectorRead.getMajorType()), callBack);
         final TransferPair tp = vvIn.makeTransferPair(vvOut);
+        memoryManager.addTransferField(vvIn, TypedFieldId.getPath(id, incomingBatch));
         transfers.add(tp);
         transferFieldIds.add(vectorRead.getFieldId().getFieldIds()[0]);
       } else if (expr instanceof DrillFuncHolderExpr &&
@@ -489,6 +537,7 @@ private void setupNewSchemaFromInput(RecordBatch incomingBatch) throws SchemaCha
 
         // save the field reference for later for getting schema when input is empty
         complexFieldReferencesList.add(namedExpression.getRef());
+        memoryManager.addComplexField(null); // this will just add an estimate to the row width
       } else {
         // need to do evaluation.
         final ValueVector vector = container.addOrGet(outputField, callBack);
@@ -497,17 +546,18 @@ private void setupNewSchemaFromInput(RecordBatch incomingBatch) throws SchemaCha
         final boolean useSetSafe = !(vector instanceof FixedWidthVector);
         final ValueVectorWriteExpression write = new ValueVectorWriteExpression(fid, expr, useSetSafe);
         final HoldingContainer hc = cg.addExpr(write, ClassGenerator.BlkCreateMode.TRUE_IF_BOUND);
+        memoryManager.addNewField(vector, write);
 
         // We cannot do multiple transfers from the same vector. However we still need to instantiate the output vector.
         if (expr instanceof ValueVectorReadExpression) {
           final ValueVectorReadExpression vectorRead = (ValueVectorReadExpression) expr;
           if (!vectorRead.hasReadPath()) {
             final TypedFieldId id = vectorRead.getFieldId();
-            final ValueVector vvIn = incomingBatch.getValueAccessorById(id.getIntermediateClass(), id.getFieldIds()).getValueVector();
+            final ValueVector vvIn = incomingBatch.getValueAccessorById(id.getIntermediateClass(),
+                    id.getFieldIds()).getValueVector();
             vvIn.makeTransferPair(vector);
           }
         }
-        logger.debug("Added eval for project expression.");
       }
     }
 
@@ -515,12 +565,16 @@ private void setupNewSchemaFromInput(RecordBatch incomingBatch) throws SchemaCha
       CodeGenerator<Projector> codeGen = cg.getCodeGenerator();
       codeGen.plainJavaCapable(true);
       // Uncomment out this line to debug the generated code.
-      // codeGen.saveCodeForDebugging(true);
+      //codeGen.saveCodeForDebugging(true);
       this.projector = context.getImplementationClass(codeGen);
       projector.setup(context, incomingBatch, this, transfers);
     } catch (ClassTransformationException | IOException e) {
       throw new SchemaChangeException("Failure while attempting to load generated class", e);
     }
+
+    long setupNewSchemaEndTime = System.currentTimeMillis();
+      logger.trace("setupNewSchemaFromInput: time {}  ms, Project {}, incoming {}",
+                  (setupNewSchemaEndTime - setupNewSchemaStartTime), this, incomingBatch);
   }
 
   @Override
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/Projector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/Projector.java
index 455d6435da..5e5e7ebe55 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/Projector.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/Projector.java
@@ -17,18 +17,18 @@
  */
 package org.apache.drill.exec.physical.impl.project;
 
-import java.util.List;
-
 import org.apache.drill.exec.compile.TemplateClassDefinition;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.TransferPair;
 
+import java.util.List;
+
 public interface Projector {
 
   public abstract void setup(FragmentContext context, RecordBatch incoming,  RecordBatch outgoing, List<TransferPair> transfers)  throws SchemaChangeException;
-  public abstract int projectRecords(int startIndex, int recordCount, int firstOutputIndex);
+  public abstract int projectRecords(RecordBatch incomingBatch, int startIndex, int recordCount, int firstOutputIndex);
 
   public static TemplateClassDefinition<Projector> TEMPLATE_DEFINITION = new TemplateClassDefinition<Projector>(Projector.class, ProjectorTemplate.class);
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java
index 499e55b41f..ec409677e3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java
@@ -17,10 +17,7 @@
  */
 package org.apache.drill.exec.physical.impl.project;
 
-import java.util.List;
-
-import javax.inject.Named;
-
+import com.google.common.collect.ImmutableList;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
@@ -29,7 +26,8 @@
 import org.apache.drill.exec.record.selection.SelectionVector2;
 import org.apache.drill.exec.record.selection.SelectionVector4;
 
-import com.google.common.collect.ImmutableList;
+import javax.inject.Named;
+import java.util.List;
 
 public abstract class ProjectorTemplate implements Projector {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ProjectorTemplate.class);
@@ -43,7 +41,9 @@ public ProjectorTemplate() {
   }
 
   @Override
-  public final int projectRecords(int startIndex, final int recordCount, int firstOutputIndex) {
+  public final int projectRecords(RecordBatch incomingRecordBatch, int startIndex, final int recordCount,
+                                  int firstOutputIndex) {
+    assert incomingRecordBatch != this; // mixed up incoming and outgoing batches?
     switch (svMode) {
     case FOUR_BYTE:
       throw new UnsupportedOperationException();
@@ -69,7 +69,8 @@ public final int projectRecords(int startIndex, final int recordCount, int first
           throw new UnsupportedOperationException(e);
         }
       }
-      if (i < startIndex + recordCount || startIndex > 0) {
+      final int totalBatchRecordCount = incomingRecordBatch.getRecordCount();
+      if (startIndex + recordCount < totalBatchRecordCount || startIndex > 0 ) {
         for (TransferPair t : transfers) {
           t.splitAndTransfer(startIndex, i - startIndex);
         }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/AbstractSV2Copier.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/AbstractSV2Copier.java
index d9f1c8ea12..321d9a87d6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/AbstractSV2Copier.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/AbstractSV2Copier.java
@@ -34,7 +34,6 @@ public void setup(RecordBatch incoming, VectorContainer outgoing) throws SchemaC
     this.sv2 = incoming.getSelectionVector2();
 
     final int count = outgoing.getNumberOfColumns();
-
     vvIn = new ValueVector[count];
 
     {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/AbstractSV4Copier.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/AbstractSV4Copier.java
index 4f3afc36ce..cd6af07b37 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/AbstractSV4Copier.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/AbstractSV4Copier.java
@@ -22,10 +22,12 @@
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.record.selection.SelectionVector4;
-import org.apache.drill.exec.vector.ValueVector;
 
 public abstract class AbstractSV4Copier extends AbstractCopier {
-  protected ValueVector[][] vvIn;
+  // Storing VectorWrapper reference instead of ValueVector[]. With EMIT outcome support underlying operator
+  // operator can generate multiple output batches with no schema changes which will change the ValueVector[]
+  // reference but not VectorWrapper reference.
+  protected VectorWrapper<?>[] vvIn;
   private SelectionVector4 sv4;
 
   @Override
@@ -34,14 +36,13 @@ public void setup(RecordBatch incoming, VectorContainer outgoing) throws SchemaC
     this.sv4 = incoming.getSelectionVector4();
 
     final int count = outgoing.getNumberOfColumns();
-
-    vvIn = new ValueVector[count][];
+    vvIn = new VectorWrapper[count];
 
     {
       int index = 0;
 
       for (VectorWrapper vectorWrapper: incoming) {
-        vvIn[index] = vectorWrapper.getValueVectors();
+        vvIn[index] = vectorWrapper;
         index++;
       }
     }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/GenericSV4Copier.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/GenericSV4Copier.java
index f9b153d050..1f3d28bdc4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/GenericSV4Copier.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/GenericSV4Copier.java
@@ -30,7 +30,8 @@ public void copyEntry(int inIndex, int outIndex) throws SchemaChangeException {
     int inOffset = inIndex & 0xFFFF;
     int inVector = inIndex >>> 16;
     for ( int i = 0;  i < vvIn.length;  i++ ) {
-      vvOut[i].copyEntry(outIndex, vvIn[i][inVector], inOffset);
+      ValueVector[] vectorsFromIncoming = vvIn[i].getValueVectors();
+      vvOut[i].copyEntry(outIndex, vectorsFromIncoming[inVector], inOffset);
     }
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java
index b4d0e7726d..36b9c9ba86 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java
@@ -27,6 +27,7 @@
 import org.apache.drill.common.expression.SchemaPath;
 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.exception.ClassTransformationException;
 import org.apache.drill.exec.exception.OutOfMemoryException;
 import org.apache.drill.exec.exception.SchemaChangeException;
@@ -38,8 +39,11 @@
 import org.apache.drill.exec.physical.config.UnionAll;
 import org.apache.drill.exec.record.AbstractBinaryRecordBatch;
 import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.JoinBatchMemoryManager;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.RecordBatchMemoryManager;
+import org.apache.drill.exec.record.RecordBatchSizer;
 import org.apache.drill.exec.record.TransferPair;
 import org.apache.drill.exec.record.TypedFieldId;
 import org.apache.drill.exec.record.VectorAccessibleUtilities;
@@ -68,6 +72,11 @@
 
   public UnionAllRecordBatch(UnionAll config, List<RecordBatch> children, FragmentContext context) throws OutOfMemoryException {
     super(config, context, true, children.get(0), children.get(1));
+
+    // get the output batch size from config.
+    int configuredBatchSize = (int) context.getOptions().getOption(ExecConstants.OUTPUT_BATCH_SIZE_VALIDATOR);
+    batchMemoryManager = new RecordBatchMemoryManager(numInputs, configuredBatchSize);
+    logger.debug("BATCH_STATS, configured output batch size: {}", configuredBatchSize);
   }
 
   @Override
@@ -106,9 +115,9 @@ public IterOutcome innerNext() {
           return IterOutcome.NONE;
         }
 
-        Pair<IterOutcome, RecordBatch> nextBatch = unionInputIterator.next();
+        Pair<IterOutcome, BatchStatusWrappper> nextBatch = unionInputIterator.next();
         IterOutcome upstream = nextBatch.left;
-        RecordBatch incoming = nextBatch.right;
+        BatchStatusWrappper batchStatus = nextBatch.right;
 
         switch (upstream) {
         case NONE:
@@ -116,14 +125,14 @@ public IterOutcome innerNext() {
         case STOP:
           return upstream;
         case OK_NEW_SCHEMA:
-          return doWork(nextBatch.right, true);
+          return doWork(batchStatus, true);
         case OK:
           // skip batches with same schema as the previous one yet having 0 row.
-          if (incoming.getRecordCount() == 0) {
-            VectorAccessibleUtilities.clear(incoming);
+          if (batchStatus.batch.getRecordCount() == 0) {
+            VectorAccessibleUtilities.clear(batchStatus.batch);
             continue;
           }
-          return doWork(nextBatch.right, false);
+          return doWork(batchStatus, false);
         default:
           throw new IllegalStateException(String.format("Unknown state %s.", upstream));
         }
@@ -142,19 +151,30 @@ public int getRecordCount() {
 
 
   @SuppressWarnings("resource")
-  private IterOutcome doWork(RecordBatch inputBatch, boolean newSchema) throws ClassTransformationException, IOException, SchemaChangeException {
-    Preconditions.checkArgument(inputBatch.getSchema().getFieldCount() == container.getSchema().getFieldCount(),
+  private IterOutcome doWork(BatchStatusWrappper batchStatus, boolean newSchema) throws ClassTransformationException, IOException, SchemaChangeException {
+    Preconditions.checkArgument(batchStatus.batch.getSchema().getFieldCount() == container.getSchema().getFieldCount(),
         "Input batch and output batch have different field counthas!");
 
     if (newSchema) {
-      createUnionAller(inputBatch);
+      createUnionAller(batchStatus.batch);
     }
 
+    // Get number of records to include in the batch.
+    final int recordsToProcess = Math.min(batchMemoryManager.getOutputRowCount(), batchStatus.getRemainingRecords());
+
     container.zeroVectors();
-    VectorUtil.allocateVectors(allocationVectors, inputBatch.getRecordCount());
-    recordCount = unionall.unionRecords(0, inputBatch.getRecordCount(), 0);
+    batchMemoryManager.allocateVectors(allocationVectors, recordsToProcess);
+    recordCount = unionall.unionRecords(batchStatus.recordsProcessed, recordsToProcess, 0);
     VectorUtil.setValueCount(allocationVectors, recordCount);
 
+    // save number of records processed so far in batch status.
+    batchStatus.recordsProcessed += recordCount;
+    batchMemoryManager.updateOutgoingStats(recordCount);
+
+    if (logger.isDebugEnabled()) {
+      logger.debug("BATCH_STATS, outgoing: {}", new RecordBatchSizer(this));
+    }
+
     if (callBack.getSchemaChangedAndReset()) {
       return IterOutcome.OK_NEW_SCHEMA;
     } else {
@@ -168,6 +188,7 @@ private void createUnionAller(RecordBatch inputBatch) throws ClassTransformation
 
     final ClassGenerator<UnionAller> cg = CodeGenerator.getRoot(UnionAller.TEMPLATE_DEFINITION, context.getOptions());
     cg.getCodeGenerator().plainJavaCapable(true);
+    // cg.getCodeGenerator().saveCodeForDebugging(true);
 
     int index = 0;
     for(VectorWrapper<?> vw : inputBatch) {
@@ -303,16 +324,25 @@ private static boolean hasSameTypeAndMode(MaterializedField leftField, Materiali
     final RecordBatch batch;
     final int inputIndex;
     final IterOutcome outcome;
+    int recordsProcessed;
+    int totalRecordsToProcess;
 
     BatchStatusWrappper(boolean prefetched, IterOutcome outcome, RecordBatch batch, int inputIndex) {
       this.prefetched = prefetched;
       this.outcome = outcome;
       this.batch = batch;
       this.inputIndex = inputIndex;
+      this.totalRecordsToProcess = batch.getRecordCount();
+      this.recordsProcessed = 0;
     }
+
+    public int getRemainingRecords() {
+      return (totalRecordsToProcess - recordsProcessed);
+    }
+
   }
 
-  private class UnionInputIterator implements Iterator<Pair<IterOutcome, RecordBatch>> {
+  private class UnionInputIterator implements Iterator<Pair<IterOutcome, BatchStatusWrappper>> {
     private Stack<BatchStatusWrappper> batchStatusStack = new Stack<>();
 
     UnionInputIterator(IterOutcome leftOutCome, RecordBatch left, IterOutcome rightOutCome, RecordBatch right) {
@@ -331,23 +361,39 @@ public boolean hasNext() {
     }
 
     @Override
-    public Pair<IterOutcome, RecordBatch> next() {
+    public Pair<IterOutcome, BatchStatusWrappper> next() {
       while (!batchStatusStack.isEmpty()) {
         BatchStatusWrappper topStatus = batchStatusStack.peek();
 
         if (topStatus.prefetched) {

  (This diff was longer than 20,000 lines, and has been truncated...)


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services