You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by vi...@apache.org on 2018/06/22 21:14:31 UTC

[drill] branch master updated (b447260 -> b92f599)

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

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


    from b447260  DRILL-6212: Prevent recursive cast expressions
     new 502d297  DRILL-5188: Expand sub-queries using rules
     new fc825c5  DRILL-6470: Remove defunct repository
     new 9299fcc  DRILL-6491: Prevent merge join for full outer join at planning stage
     new 9c7e557  DRILL-6502: Rename CorrelatePrel to LateralJoinPrel.
     new 6823a8f  DRILL-6513: Max query memory per node set to Drillbit's maximum direct memory
     new aa127b7  DRILL-6523: Fix NPE for describe of partial schema
     new 947e6e5  DRILL-6486: BitVector split and transfer does not work correctly for non byte-multiple transfer lengths
     new 67d992c  DRILL-6340 Output Batch Control in Project using the RecordBatchSizer
     new b92f599  DRILL-6454: Native MapR DB plugin support for Hive MapR-DB json table

The 9 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../scanner/persistence/AnnotationDescriptor.java  |   2 +
 .../drill/exec/store/mapr/TableFormatPlugin.java   |  11 +-
 .../exec/store/mapr/db/MapRDBFormatMatcher.java    |  18 +-
 .../drill/exec/store/mapr/db/MapRDBGroupScan.java  |   8 +-
 .../store/mapr/db/MapRDBPushFilterIntoScan.java    |   2 +-
 .../store/mapr/db/binary/BinaryTableGroupScan.java |   8 +-
 .../store/mapr/db/json/JsonTableGroupScan.java     |   6 +-
 contrib/storage-hive/core/pom.xml                  |   5 +
 ...ertHiveMapRDBJsonScanToDrillMapRDBJsonScan.java | 186 +++++++++++
 .../ConvertHiveParquetScanToDrillParquetScan.java  | 116 +------
 .../exec/store/hive/HiveMetadataProvider.java      |   2 +-
 .../drill/exec/store/hive/HiveStoragePlugin.java   |  16 +-
 .../drill/exec/store/hive/HiveTableWrapper.java    |   7 +-
 .../drill/exec/store/hive/HiveUtilities.java       | 149 +++++++--
 .../exec/TestHiveDrillNativeParquetReader.java     |   4 +-
 exec/java-exec/src/main/codegen/config.fmpp        |   1 +
 .../src/main/codegen/data/SingleValue.tdd          |  62 ++++
 .../src/main/codegen/templates/SingleValueAgg.java | 144 +++++++++
 .../java/org/apache/drill/exec/ExecConstants.java  |  11 +-
 .../drill/exec/expr/AbstractExecExprVisitor.java   |  58 ++++
 .../drill/exec/expr/ValueVectorReadExpression.java |   8 +-
 .../exec/expr/ValueVectorWriteExpression.java      |   9 +-
 .../exec/expr/annotations/FunctionTemplate.java    |  34 ++
 .../apache/drill/exec/expr/fn/DrillFuncHolder.java |  29 +-
 .../drill/exec/expr/fn/FunctionAttributes.java     |   8 +
 .../drill/exec/expr/fn/impl/ByteSubstring.java     |   3 +-
 .../exec/expr/fn/impl/CastVarCharVar16Char.java    |   3 +-
 .../drill/exec/expr/fn/impl/CharSubstring.java     |   3 +-
 .../drill/exec/expr/fn/impl/ContextFunctions.java  |  10 +-
 .../drill/exec/expr/fn/impl/DateTypeFunctions.java |   4 +-
 .../exec/expr/fn/impl/SimpleCastFunctions.java     |   3 +-
 .../drill/exec/expr/fn/impl/StringFunctions.java   | 108 ++++---
 .../exec/expr/fn/impl/conv/BigIntBEConvertTo.java  |   4 +-
 .../exec/expr/fn/impl/conv/BigIntConvertTo.java    |   4 +-
 .../expr/fn/impl/conv/BigIntVLongConvertTo.java    |   4 +-
 .../expr/fn/impl/conv/BooleanByteConvertTo.java    |   4 +-
 .../expr/fn/impl/conv/DateEpochBEConvertTo.java    |   4 +-
 .../exec/expr/fn/impl/conv/DateEpochConvertTo.java |   4 +-
 .../exec/expr/fn/impl/conv/DoubleBEConvertTo.java  |   4 +-
 .../exec/expr/fn/impl/conv/DoubleConvertTo.java    |   4 +-
 .../exec/expr/fn/impl/conv/DummyConvertFrom.java   |   3 +-
 .../exec/expr/fn/impl/conv/DummyConvertTo.java     |   3 +-
 .../exec/expr/fn/impl/conv/FloatBEConvertTo.java   |   4 +-
 .../exec/expr/fn/impl/conv/FloatConvertTo.java     |   4 +-
 .../exec/expr/fn/impl/conv/IntBEConvertTo.java     |   4 +-
 .../drill/exec/expr/fn/impl/conv/IntConvertTo.java |   4 +-
 .../exec/expr/fn/impl/conv/IntVIntConvertTo.java   |   4 +-
 .../exec/expr/fn/impl/conv/JsonConvertTo.java      |   6 +-
 .../expr/fn/impl/conv/SmallIntBEConvertTo.java     |   4 +-
 .../exec/expr/fn/impl/conv/SmallIntConvertTo.java  |   4 +-
 .../expr/fn/impl/conv/TimeEpochBEConvertTo.java    |   4 +-
 .../exec/expr/fn/impl/conv/TimeEpochConvertTo.java |   4 +-
 .../fn/impl/conv/TimeStampEpochBEConvertTo.java    |   4 +-
 .../exec/expr/fn/impl/conv/TinyIntConvertTo.java   |   4 +-
 .../exec/expr/fn/impl/conv/UInt4BEConvertTo.java   |   4 +-
 .../exec/expr/fn/impl/conv/UInt4ConvertTo.java     |   4 +-
 .../exec/expr/fn/impl/conv/UInt8ConvertTo.java     |   4 +-
 .../exec/expr/fn/impl/conv/UTF16ConvertFrom.java   |   3 +-
 .../exec/expr/fn/impl/conv/UTF16ConvertTo.java     |   4 +-
 .../exec/expr/fn/impl/conv/UTF8ConvertFrom.java    |   3 +-
 .../exec/expr/fn/impl/conv/UTF8ConvertTo.java      |   3 +-
 ...peInference.java => OutputWidthCalculator.java} |   9 +-
 .../expr/fn/output/OutputWidthCalculators.java     | 113 +++++++
 .../drill/exec/physical/impl/join/JoinUtils.java   | 105 ++++++-
 .../impl/project/OutputSizeEstimateConstants.java  |  46 +++
 .../impl/project/OutputWidthExpression.java        | 147 +++++++++
 .../physical/impl/project/OutputWidthVisitor.java  | 278 ++++++++++++++++
 .../impl/project/OutputWidthVisitorState.java}     |  28 +-
 .../impl/project/ProjectMemoryManager.java         | 310 ++++++++++++++++++
 .../physical/impl/project/ProjectRecordBatch.java  |  84 ++++-
 .../exec/physical/impl/project/Projector.java      |   6 +-
 .../physical/impl/project/ProjectorTemplate.java   |  15 +-
 .../apache/drill/exec/planner/PlannerPhase.java    |  14 +-
 .../apache/drill/exec/planner/RuleInstance.java    |  10 +
 ...teRelBase.java => DrillLateralJoinRelBase.java} |   6 +-
 .../exec/planner/logical/DrillCorrelateRule.java   |   6 +-
 ...lCorrelateRel.java => DrillLateralJoinRel.java} |  10 +-
 .../exec/planner/logical/PreProcessLogicalRel.java |  18 +-
 .../{CorrelatePrel.java => LateralJoinPrel.java}   |  14 +-
 .../{CorrelatePrule.java => LateralJoinPrule.java} |  25 +-
 .../drill/exec/planner/physical/MergeJoinPrel.java |   3 +-
 .../drill/exec/planner/physical/UnnestPrel.java    |   2 +-
 .../physical/explain/NumberingRelWriter.java       |  20 +-
 .../planner/physical/visitor/BasePrelVisitor.java  |   4 +-
 .../visitor/ExcessiveExchangeIdentifier.java       |   6 +-
 .../physical/visitor/JoinPrelRenameVisitor.java    |   8 +-
 .../exec/planner/physical/visitor/PrelVisitor.java |  22 +-
 .../physical/visitor/PrelVisualizerVisitor.java    |   4 +-
 .../drill/exec/planner/sql/SqlConverter.java       |   8 +-
 .../planner/sql/handlers/DefaultSqlHandler.java    |  12 +-
 .../sql/handlers/DescribeSchemaHandler.java        |  53 ++--
 .../apache/drill/exec/record/RecordBatchSizer.java |  83 +++--
 .../org/apache/drill/exec/record/TypedFieldId.java |  32 ++
 .../exec/server/options/SystemOptionManager.java   |   2 +
 .../java-exec/src/main/resources/drill-module.conf |   2 +
 .../java/org/apache/drill/TestCorrelation.java     |  46 +++
 .../apache/drill/TestDisabledFunctionality.java    |  37 ---
 .../java/org/apache/drill/TestExampleQueries.java  |  33 ++
 .../java/org/apache/drill/TestTpchDistributed.java |   1 -
 .../apache/drill/TestTpchDistributedStreaming.java |   1 -
 .../java/org/apache/drill/TestTpchExplain.java     |   1 -
 .../test/java/org/apache/drill/TestTpchLimit0.java |   1 -
 .../java/org/apache/drill/TestTpchPlanning.java    |   1 -
 .../java/org/apache/drill/TestTpchSingleMode.java  |   1 -
 .../exec/compile/TestLargeFileCompilation.java     |   2 +
 .../drill/exec/fn/impl/TestAggregateFunctions.java |  80 +++++
 .../physical/impl/join/TestMergeJoinAdvanced.java  | 160 ++++++----
 .../impl/lateraljoin/TestLateralPlans.java         |   6 +-
 .../exec/physical/unit/TestOutputBatchSize.java    | 350 +++++++++++++++++++++
 .../drill/exec/record/vector/TestValueVector.java  |   3 +
 .../org/apache/drill/exec/sql/TestInfoSchema.java  |   8 +-
 .../drill/exec/vector/TestSplitAndTransfer.java    | 108 ++++++-
 .../org/apache/drill/exec/vector/BitVector.java    |  45 ++-
 pom.xml                                            |  14 +-
 114 files changed, 2964 insertions(+), 590 deletions(-)
 create mode 100644 contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/logical/ConvertHiveMapRDBJsonScanToDrillMapRDBJsonScan.java
 create mode 100644 exec/java-exec/src/main/codegen/data/SingleValue.tdd
 create mode 100644 exec/java-exec/src/main/codegen/templates/SingleValueAgg.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/expr/AbstractExecExprVisitor.java
 copy exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/output/{ReturnTypeInference.java => OutputWidthCalculator.java} (77%)
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/output/OutputWidthCalculators.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/OutputSizeEstimateConstants.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/OutputWidthExpression.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/OutputWidthVisitor.java
 copy exec/java-exec/src/main/java/org/apache/drill/exec/{schema/BackedRecord.java => physical/impl/project/OutputWidthVisitorState.java} (59%)
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectMemoryManager.java
 rename exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/{DrillCorrelateRelBase.java => DrillLateralJoinRelBase.java} (87%)
 rename exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/{DrillCorrelateRel.java => DrillLateralJoinRel.java} (80%)
 rename exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/{CorrelatePrel.java => LateralJoinPrel.java} (88%)
 rename exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/{CorrelatePrule.java => LateralJoinPrule.java} (69%)


[drill] 01/09: DRILL-5188: Expand sub-queries using rules

Posted by vi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 502d2977092eecda0a4aa0482b5f96459c315227
Author: Volodymyr Vysotskyi <vv...@gmail.com>
AuthorDate: Fri May 18 15:54:16 2018 +0300

    DRILL-5188: Expand sub-queries using rules
    
    - Add check for agg with group by literal
    - Allow NLJ for limit 1
    - Implement single_value aggregate function
    
    closes #1321
---
 exec/java-exec/src/main/codegen/config.fmpp        |   1 +
 .../src/main/codegen/data/SingleValue.tdd          |  62 +++++++++
 .../src/main/codegen/templates/SingleValueAgg.java | 144 +++++++++++++++++++++
 .../drill/exec/physical/impl/join/JoinUtils.java   | 105 ++++++++++++++-
 .../apache/drill/exec/planner/PlannerPhase.java    |  10 ++
 .../apache/drill/exec/planner/RuleInstance.java    |  10 ++
 .../exec/planner/logical/PreProcessLogicalRel.java |  18 +--
 .../drill/exec/planner/sql/SqlConverter.java       |   8 +-
 .../planner/sql/handlers/DefaultSqlHandler.java    |  12 +-
 .../java/org/apache/drill/TestCorrelation.java     |  46 +++++++
 .../apache/drill/TestDisabledFunctionality.java    |  37 ------
 .../java/org/apache/drill/TestExampleQueries.java  |  33 +++++
 .../java/org/apache/drill/TestTpchDistributed.java |   1 -
 .../apache/drill/TestTpchDistributedStreaming.java |   1 -
 .../java/org/apache/drill/TestTpchExplain.java     |   1 -
 .../test/java/org/apache/drill/TestTpchLimit0.java |   1 -
 .../java/org/apache/drill/TestTpchPlanning.java    |   1 -
 .../java/org/apache/drill/TestTpchSingleMode.java  |   1 -
 .../drill/exec/fn/impl/TestAggregateFunctions.java |  80 ++++++++++++
 pom.xml                                            |   2 +-
 20 files changed, 504 insertions(+), 70 deletions(-)

diff --git a/exec/java-exec/src/main/codegen/config.fmpp b/exec/java-exec/src/main/codegen/config.fmpp
index 50f110d..e233974 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/SingleValue.tdd b/exec/java-exec/src/main/codegen/data/SingleValue.tdd
new file mode 100644
index 0000000..a42fe3b
--- /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/SingleValueAgg.java b/exec/java-exec/src/main/codegen/templates/SingleValueAgg.java
new file mode 100644
index 0000000..c0ff6cf
--- /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/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 e4dab91..b974537 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.expression.LogicalExpression;
 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 class JoinUtils {
       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 class JoinUtils {
       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 class JoinUtils {
     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/planner/PlannerPhase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java
index 2a79751..b78d76c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java
@@ -113,6 +113,16 @@ public enum PlannerPhase {
     }
   },
 
+  SUBQUERY_REWRITE("Sub-queries rewrites") {
+    public RuleSet getRules(OptimizerRulesContext context, Collection<StoragePlugin> plugins) {
+      return RuleSets.ofList(
+          RuleInstance.SUB_QUERY_FILTER_REMOVE_RULE,
+          RuleInstance.SUB_QUERY_PROJECT_REMOVE_RULE,
+          RuleInstance.SUB_QUERY_JOIN_REMOVE_RULE
+      );
+    }
+  },
+
   LOGICAL_PRUNE("Logical Planning (with partition pruning)") {
     public RuleSet getRules(OptimizerRulesContext context, Collection<StoragePlugin> plugins) {
       return PlannerPhase.mergedRuleSets(
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/RuleInstance.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/RuleInstance.java
index 80bbe88..8aec96c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/RuleInstance.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/RuleInstance.java
@@ -40,6 +40,7 @@ import org.apache.calcite.rel.rules.ProjectToWindowRule;
 import org.apache.calcite.rel.rules.ProjectWindowTransposeRule;
 import org.apache.calcite.rel.rules.ReduceExpressionsRule;
 import org.apache.calcite.rel.rules.SortRemoveRule;
+import org.apache.calcite.rel.rules.SubQueryRemoveRule;
 import org.apache.calcite.rel.rules.UnionToDistinctRule;
 import org.apache.drill.exec.planner.logical.DrillConditions;
 import org.apache.drill.exec.planner.logical.DrillRelFactories;
@@ -130,4 +131,13 @@ public interface RuleInstance {
 
   FilterRemoveIsNotDistinctFromRule REMOVE_IS_NOT_DISTINCT_FROM_RULE =
       new FilterRemoveIsNotDistinctFromRule(DrillRelBuilder.proto(DrillRelFactories.DRILL_LOGICAL_FILTER_FACTORY));
+
+  SubQueryRemoveRule SUB_QUERY_FILTER_REMOVE_RULE =
+      new SubQueryRemoveRule.SubQueryFilterRemoveRule(DrillRelFactories.LOGICAL_BUILDER);
+
+  SubQueryRemoveRule SUB_QUERY_PROJECT_REMOVE_RULE =
+      new SubQueryRemoveRule.SubQueryProjectRemoveRule(DrillRelFactories.LOGICAL_BUILDER);
+
+  SubQueryRemoveRule SUB_QUERY_JOIN_REMOVE_RULE =
+      new SubQueryRemoveRule.SubQueryJoinRemoveRule(DrillRelFactories.LOGICAL_BUILDER);
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/PreProcessLogicalRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/PreProcessLogicalRel.java
index f3c6ce0..cd696ea 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/PreProcessLogicalRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/PreProcessLogicalRel.java
@@ -32,8 +32,6 @@ import org.apache.drill.exec.planner.sql.DrillOperatorTable;
 import org.apache.drill.exec.planner.sql.parser.DrillCalciteWrapperUtility;
 import org.apache.drill.exec.util.ApproximateStringMatcher;
 import org.apache.drill.exec.work.foreman.SqlUnsupportedException;
-import org.apache.calcite.rel.core.AggregateCall;
-import org.apache.calcite.rel.logical.LogicalAggregate;
 import org.apache.calcite.rel.logical.LogicalProject;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelShuttleImpl;
@@ -46,7 +44,6 @@ import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.sql.SqlFunction;
 import org.apache.calcite.sql.SqlOperator;
-import org.apache.calcite.sql.fun.SqlSingleValueAggFunction;
 import org.apache.calcite.util.NlsString;
 
 /**
@@ -79,19 +76,6 @@ public class PreProcessLogicalRel extends RelShuttleImpl {
   }
 
   @Override
-  public RelNode visit(LogicalAggregate aggregate) {
-    for(AggregateCall aggregateCall : aggregate.getAggCallList()) {
-      if(aggregateCall.getAggregation() instanceof SqlSingleValueAggFunction) {
-        unsupportedOperatorCollector.setException(SqlUnsupportedException.ExceptionType.FUNCTION,
-            "Non-scalar sub-query used in an expression\n" +
-            "See Apache Drill JIRA: DRILL-1937");
-        throw new UnsupportedOperationException();
-      }
-    }
-    return visitChild(aggregate, 0, aggregate.getInput());
-  }
-
-  @Override
   public RelNode visit(LogicalProject project) {
     final List<RexNode> projExpr = Lists.newArrayList();
     for(RexNode rexNode : project.getChildExps()) {
@@ -168,7 +152,7 @@ public class PreProcessLogicalRel extends RelShuttleImpl {
       exprList.add(newExpr);
     }
 
-    if (rewrite == true) {
+    if (rewrite) {
       LogicalProject newProject = project.copy(project.getTraitSet(), project.getInput(0), exprList, project.getRowType());
       return visitChild(newProject, 0, project.getInput());
     }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java
index b8659d1..3f65ad2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java
@@ -64,7 +64,6 @@ import org.apache.calcite.sql.validate.SqlValidatorCatalogReader;
 import org.apache.calcite.sql.validate.SqlValidatorImpl;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
-import org.apache.calcite.sql2rel.RelDecorrelator;
 import org.apache.calcite.sql2rel.SqlToRelConverter;
 import org.apache.calcite.tools.RelBuilderFactory;
 import org.apache.calcite.util.Util;
@@ -384,10 +383,7 @@ public class SqlConverter {
     //To avoid unexpected column errors set a value of top to false
     final RelRoot rel = sqlToRelConverter.convertQuery(validatedNode, false, false);
     final RelRoot rel2 = rel.withRel(sqlToRelConverter.flattenTypes(rel.rel, true));
-    final RelRoot rel3 = rel2.withRel(
-        RelDecorrelator.decorrelateQuery(rel2.rel,
-            sqlToRelConverterConfig.getRelBuilderFactory().create(cluster, null)));
-    return rel3;
+    return rel2;
   }
 
   private class Expander implements RelOptTable.ViewExpander {
@@ -478,7 +474,7 @@ public class SqlConverter {
 
     @Override
     public boolean isExpand() {
-      return SqlToRelConverterConfig.DEFAULT.isExpand();
+      return false;
     }
 
     @Override
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
index 98e017f..1e671ff 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
@@ -55,6 +55,7 @@ import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.sql.SqlExplainLevel;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
+import org.apache.calcite.sql2rel.RelDecorrelator;
 import org.apache.calcite.tools.Program;
 import org.apache.calcite.tools.Programs;
 import org.apache.calcite.tools.RelConversionException;
@@ -78,6 +79,7 @@ import org.apache.drill.exec.planner.common.DrillRelOptUtil;
 import org.apache.drill.exec.planner.cost.DrillDefaultRelMetadataProvider;
 import org.apache.drill.exec.planner.logical.DrillProjectRel;
 import org.apache.drill.exec.planner.logical.DrillRel;
+import org.apache.drill.exec.planner.logical.DrillRelFactories;
 import org.apache.drill.exec.planner.logical.DrillScreenRel;
 import org.apache.drill.exec.planner.logical.DrillStoreRel;
 import org.apache.drill.exec.planner.logical.PreProcessLogicalRel;
@@ -658,10 +660,16 @@ public class DefaultSqlHandler extends AbstractSqlHandler {
     return typedSqlNode;
   }
 
-  private RelNode convertToRel(SqlNode node) throws RelConversionException {
+  private RelNode convertToRel(SqlNode node) {
     final RelNode convertedNode = config.getConverter().toRel(node).rel;
     log("INITIAL", convertedNode, logger, null);
-    return transform(PlannerType.HEP, PlannerPhase.WINDOW_REWRITE, convertedNode);
+    RelNode transformedNode = transform(PlannerType.HEP,
+        PlannerPhase.SUBQUERY_REWRITE, convertedNode);
+
+    RelNode decorrelatedNode = RelDecorrelator.decorrelateQuery(transformedNode,
+        DrillRelFactories.LOGICAL_BUILDER.create(transformedNode.getCluster(), null));
+
+    return transform(PlannerType.HEP, PlannerPhase.WINDOW_REWRITE, decorrelatedNode);
   }
 
   private RelNode preprocessNode(RelNode rel) throws SqlUnsupportedException {
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestCorrelation.java b/exec/java-exec/src/test/java/org/apache/drill/TestCorrelation.java
index 47f0ae8..d40540e 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestCorrelation.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestCorrelation.java
@@ -58,4 +58,50 @@ public class TestCorrelation extends PlanTestBase {
       .run();
   }
 
+  @Test
+  public void testExistsScalarSubquery() throws Exception {
+    String query =
+        "SELECT employee_id\n" +
+        "FROM cp.`employee.json`\n" +
+        "WHERE EXISTS\n" +
+        "    (SELECT *\n" +
+        "     FROM cp.`employee.json` cs2\n" +
+        "     )\n" +
+        "limit 1";
+
+    testBuilder()
+        .sqlQuery(query)
+        .unOrdered()
+        .baselineColumns("employee_id")
+        .baselineValues(1L)
+        .go();
+  }
+
+  @Test
+  public void testSeveralExistsCorrelateSubquery() throws Exception {
+    String query =
+        "SELECT cs1.employee_id\n" +
+        "FROM cp.`employee.json` cs1,\n" +
+        "     cp.`employee.json` cs3\n" +
+        "WHERE cs1.hire_date = cs3.hire_date\n" +
+        "  AND EXISTS\n" +
+        "    (SELECT *\n" +
+        "     FROM cp.`employee.json` cs2\n" +
+        "     WHERE " +
+        "       cs1.position_id > cs2.position_id\n" +
+        "       AND" +
+        "       cs1.epmloyee_id = cs2.epmloyee_id" +
+        "       )\n" +
+        "  AND EXISTS\n" +
+        "    (SELECT *\n" +
+        "     FROM cp.`employee.json` cr1\n" +
+        "     WHERE cs1.position_id = cr1.position_id)\n" +
+        "LIMIT 1";
+
+    testBuilder()
+      .sqlQuery(query)
+      .unOrdered()
+      .expectsEmptyResultSet()
+      .go();
+  }
 }
\ No newline at end of file
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestDisabledFunctionality.java b/exec/java-exec/src/test/java/org/apache/drill/TestDisabledFunctionality.java
index 781fce3..679d01e 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestDisabledFunctionality.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestDisabledFunctionality.java
@@ -47,30 +47,6 @@ public class TestDisabledFunctionality extends BaseTestQuery {
     throw ex;
   }
 
-  @Test(expected = UnsupportedFunctionException.class)  // see DRILL-1937
-  public void testDisabledExplainplanForComparisonWithNonscalarSubquery() throws Exception {
-    try {
-      test("explain plan for select n_name from cp.`tpch/nation.parquet` " +
-           "where n_nationkey = " +
-           "(select r_regionkey from cp.`tpch/region.parquet` " +
-           "where r_regionkey = 1)");
-    } catch(UserException ex) {
-      throwAsUnsupportedException(ex);
-    }
-  }
-
-  @Test(expected = UnsupportedFunctionException.class)  // see DRILL-1937
-  public void testDisabledComparisonWithNonscalarSubquery() throws Exception {
-    try {
-      test("select n_name from cp.`tpch/nation.parquet` " +
-           "where n_nationkey = " +
-           "(select r_regionkey from cp.`tpch/region.parquet` " +
-           "where r_regionkey = 1)");
-    } catch(UserException ex) {
-      throwAsUnsupportedException(ex);
-    }
-  }
-
   @Test(expected = UnsupportedRelOperatorException.class) // see DRILL-1921
   public void testDisabledIntersect() throws Exception {
     try {
@@ -215,19 +191,6 @@ public class TestDisabledFunctionality extends BaseTestQuery {
     }
   }
 
-  @Test(expected = UnsupportedFunctionException.class) // see DRILL-1325, DRILL-2155, see DRILL-1937
-  public void testMultipleUnsupportedOperatorations() throws Exception {
-    try {
-      test("select a.lastname, b.n_name " +
-          "from cp.`employee.json` a, cp.`tpch/nation.parquet` b " +
-          "where b.n_nationkey = " +
-          "(select r_regionkey from cp.`tpch/region.parquet` " +
-          "where r_regionkey = 1)");
-    } catch(UserException ex) {
-      throwAsUnsupportedException(ex);
-    }
-  }
-
   @Test(expected = UnsupportedRelOperatorException.class) // see DRILL-2068, DRILL-1325
   public void testExplainPlanForCartesianJoin() throws Exception {
     try {
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestExampleQueries.java b/exec/java-exec/src/test/java/org/apache/drill/TestExampleQueries.java
index 3e17a7a..adc8e35 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestExampleQueries.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestExampleQueries.java
@@ -1165,4 +1165,37 @@ public class TestExampleQueries extends BaseTestQuery {
         .build()
         .run();
   }
+
+  @Test
+  public void testComparisonWithSingleValueSubQuery() throws Exception {
+    String query = "select n_name from cp.`tpch/nation.parquet` " +
+        "where n_nationkey = " +
+        "(select r_regionkey from cp.`tpch/region.parquet` " +
+        "where r_regionkey = 1)";
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[]{"agg.*SINGLE_VALUE", "Filter.*=\\(\\$0, 1\\)"});
+
+    testBuilder()
+        .sqlQuery(query)
+        .unOrdered()
+        .baselineColumns("n_name")
+        .baselineValues("ARGENTINA")
+        .go();
+  }
+
+  @Test
+  public void testMultipleComparisonWithSingleValueSubQuery() throws Exception {
+    String query = "select a.last_name, b.n_name " +
+        "from cp.`employee.json` a, cp.`tpch/nation.parquet` b " +
+        "where b.n_nationkey = " +
+        "(select r_regionkey from cp.`tpch/region.parquet` " +
+        "where r_regionkey = 1) limit 1";
+
+    testBuilder()
+        .sqlQuery(query)
+        .unOrdered()
+        .baselineColumns("last_name", "n_name")
+        .baselineValues("Nowmer", "ARGENTINA")
+        .go();
+  }
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestTpchDistributed.java b/exec/java-exec/src/test/java/org/apache/drill/TestTpchDistributed.java
index 54f7250..fd97564 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestTpchDistributed.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestTpchDistributed.java
@@ -138,7 +138,6 @@ public class TestTpchDistributed extends BaseTestQuery {
   }
 
   @Test
-  @Ignore
   public void tpch21() throws Exception{
     testDistributed("queries/tpch/21.sql");
   }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestTpchDistributedStreaming.java b/exec/java-exec/src/test/java/org/apache/drill/TestTpchDistributedStreaming.java
index 56287c9..ed242be 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestTpchDistributedStreaming.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestTpchDistributedStreaming.java
@@ -145,7 +145,6 @@ public class TestTpchDistributedStreaming extends BaseTestQuery {
   }
 
   @Test
-  @Ignore
   public void tpch21() throws Exception{
     testDistributed("queries/tpch/21.sql");
   }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestTpchExplain.java b/exec/java-exec/src/test/java/org/apache/drill/TestTpchExplain.java
index 3cbe5ef..1aa9de2 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestTpchExplain.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestTpchExplain.java
@@ -148,7 +148,6 @@ public class TestTpchExplain extends BaseTestQuery {
   }
 
   @Test
-  @Ignore
   public void tpch21() throws Exception{
     doExplain("queries/tpch/21.sql");
   }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestTpchLimit0.java b/exec/java-exec/src/test/java/org/apache/drill/TestTpchLimit0.java
index 9400c7c..51aea18 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestTpchLimit0.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestTpchLimit0.java
@@ -140,7 +140,6 @@ public class TestTpchLimit0 extends BaseTestQuery {
   }
 
   @Test
-  @Ignore
   public void tpch21() throws Exception{
     testLimitZero("queries/tpch/21.sql");
   }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestTpchPlanning.java b/exec/java-exec/src/test/java/org/apache/drill/TestTpchPlanning.java
index 1745301..f995ce6 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestTpchPlanning.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestTpchPlanning.java
@@ -141,7 +141,6 @@ public class TestTpchPlanning extends PlanningBase {
   }
 
   @Test
-  @Ignore // DRILL-519
   public void tpch21() throws Exception {
     testSqlPlanFromFile("queries/tpch/21.sql");
   }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestTpchSingleMode.java b/exec/java-exec/src/test/java/org/apache/drill/TestTpchSingleMode.java
index 2863bbb..a3e34c0 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestTpchSingleMode.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestTpchSingleMode.java
@@ -139,7 +139,6 @@ public class TestTpchSingleMode extends BaseTestQuery {
   }
 
   @Test
-  @Ignore
   public void tpch21() throws Exception{
     testSingleMode("queries/tpch/21.sql");
   }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestAggregateFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestAggregateFunctions.java
index 0a5b7cd..445b18e 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestAggregateFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestAggregateFunctions.java
@@ -25,6 +25,9 @@ import org.apache.drill.common.exceptions.UserRemoteException;
 import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
+import org.apache.drill.exec.record.RecordBatchLoader;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.util.Text;
 import org.apache.drill.test.BaseTestQuery;
 import org.apache.drill.categories.OperatorTest;
 import org.apache.drill.PlanTestBase;
@@ -37,23 +40,32 @@ import org.apache.drill.exec.proto.UserBitShared;
 import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.ExpectedException;
 
 import java.io.BufferedWriter;
 import java.io.File;
 import java.io.FileWriter;
 import java.math.BigDecimal;
 import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.stream.Collectors;
 
+import static org.hamcrest.CoreMatchers.containsString;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 @Category({SqlFunctionTest.class, OperatorTest.class, PlannerTest.class})
 public class TestAggregateFunctions extends BaseTestQuery {
 
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
   @BeforeClass
   public static void setupFiles() {
     dirTestWatcher.copyResourceToRoot(Paths.get("agg"));
@@ -521,6 +533,7 @@ public class TestAggregateFunctions extends BaseTestQuery {
         .go();
 
   }
+
   @Test
   public void minMaxEmptyNonNullableInput() throws Exception {
     // test min and max functions on required type
@@ -567,6 +580,73 @@ public class TestAggregateFunctions extends BaseTestQuery {
     }
   }
 
+  @Test
+  public void testSingleValueFunction() throws Exception {
+    List<String> tableNames = ImmutableList.of(
+        "cp.`parquet/alltypes_required.parquet`",
+        "cp.`parquet/alltypes_optional.parquet`");
+    for (String tableName : tableNames) {
+      final QueryDataBatch result =
+          testSqlWithResults(String.format("select * from %s limit 1", tableName)).get(0);
+
+      final Map<String, StringBuilder> functions = new HashMap<>();
+      functions.put("single_value", new StringBuilder());
+
+      final Map<String, Object> resultingValues = new HashMap<>();
+      final List<String> columns = new ArrayList<>();
+
+      final RecordBatchLoader loader = new RecordBatchLoader(getAllocator());
+      loader.load(result.getHeader().getDef(), result.getData());
+
+      for (VectorWrapper<?> vectorWrapper : loader.getContainer()) {
+        final String fieldName = vectorWrapper.getField().getName();
+        Object object = vectorWrapper.getValueVector().getAccessor().getObject(0);
+        // VarCharVector returns Text instance, but baseline values should contain String value
+        if (object instanceof Text) {
+          object = object.toString();
+        }
+        resultingValues.put(String.format("`%s`", fieldName), object);
+        for (Map.Entry<String, StringBuilder> function : functions.entrySet()) {
+          function.getValue()
+              .append(function.getKey())
+              .append("(")
+              .append(fieldName)
+              .append(") ")
+              .append(fieldName)
+              .append(",");
+        }
+        columns.add(fieldName);
+      }
+      loader.clear();
+      result.release();
+
+      String columnsList = columns.stream()
+          .collect(Collectors.joining(", "));
+
+      final List<Map<String, Object>> baselineRecords = new ArrayList<>();
+      baselineRecords.add(resultingValues);
+
+      for (StringBuilder selectBody : functions.values()) {
+        selectBody.setLength(selectBody.length() - 1);
+
+        testBuilder()
+            .sqlQuery("select %s from (select %s from %s limit 1)",
+                selectBody.toString(), columnsList, tableName)
+            .unOrdered()
+            .baselineRecords(baselineRecords)
+            .go();
+      }
+    }
+  }
+
+  @Test
+  public void testSingleValueWithMultipleValuesInput() throws Exception {
+    thrown.expect(UserRemoteException.class);
+    thrown.expectMessage(containsString("FUNCTION ERROR"));
+    thrown.expectMessage(containsString("Input for single_value function has more than one row"));
+    test("select single_value(n_name) from cp.`tpch/nation.parquet`");
+  }
+
   /*
    * Streaming agg on top of a filter produces wrong results if the first two batches are filtered out.
    * In the below test we have three files in the input directory and since the ordering of reading
diff --git a/pom.xml b/pom.xml
index 242b134..f69288e 100644
--- a/pom.xml
+++ b/pom.xml
@@ -45,7 +45,7 @@
     <dep.guava.version>18.0</dep.guava.version>
     <forkCount>2</forkCount>
     <parquet.version>1.10.0</parquet.version>
-    <calcite.version>1.16.0-drill-r3</calcite.version>
+    <calcite.version>1.16.0-drill-r4</calcite.version>
     <avatica.version>1.11.0</avatica.version>
     <janino.version>2.7.6</janino.version>
     <sqlline.version>1.1.9-drill-r7</sqlline.version>


[drill] 09/09: DRILL-6454: Native MapR DB plugin support for Hive MapR-DB json table

Posted by vi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit b92f5996eeb9bc41c23c0faddb65b248c7cc8c4f
Author: Vitalii Diravka <vi...@gmail.com>
AuthorDate: Thu May 31 20:39:44 2018 -0700

    DRILL-6454: Native MapR DB plugin support for Hive MapR-DB json table
    
    closes #1314
---
 .../drill/exec/store/mapr/TableFormatPlugin.java   |  11 +-
 .../exec/store/mapr/db/MapRDBFormatMatcher.java    |  18 +-
 .../drill/exec/store/mapr/db/MapRDBGroupScan.java  |   8 +-
 .../store/mapr/db/MapRDBPushFilterIntoScan.java    |   2 +-
 .../store/mapr/db/binary/BinaryTableGroupScan.java |   8 +-
 .../store/mapr/db/json/JsonTableGroupScan.java     |   6 +-
 contrib/storage-hive/core/pom.xml                  |   5 +
 ...ertHiveMapRDBJsonScanToDrillMapRDBJsonScan.java | 186 +++++++++++++++++++++
 .../ConvertHiveParquetScanToDrillParquetScan.java  | 116 +------------
 .../exec/store/hive/HiveMetadataProvider.java      |   2 +-
 .../drill/exec/store/hive/HiveStoragePlugin.java   |  16 +-
 .../drill/exec/store/hive/HiveTableWrapper.java    |   7 +-
 .../drill/exec/store/hive/HiveUtilities.java       | 149 ++++++++++++++---
 .../exec/TestHiveDrillNativeParquetReader.java     |   4 +-
 .../java/org/apache/drill/exec/ExecConstants.java  |   8 +
 .../exec/server/options/SystemOptionManager.java   |   2 +
 .../java-exec/src/main/resources/drill-module.conf |   2 +
 17 files changed, 378 insertions(+), 172 deletions(-)

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 0d983bd..be86b4c 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.common.logical.StoragePluginConfig;
 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 @@ public abstract class TableFormatPlugin implements FormatPlugin {
   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 abstract class TableFormatPlugin implements FormatPlugin {
     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 4ce6b7d..d4978b9 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.dfs.FormatSelection;
 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 @@ public class MapRDBFormatMatcher extends TableFormatMatcher {
   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 effdcde..927bd70 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.common.expression.SchemaPath;
 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 @@ import com.google.common.collect.Sets;
 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 abstract class MapRDBGroupScan extends AbstractGroupScan {
     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 abstract class MapRDBGroupScan extends AbstractGroupScan {
   }
 
   @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 8655f5b..e2ba238 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 @@ public abstract class MapRDBPushFilterIntoScan extends StoragePluginOptimizerRul
     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 16b979e..216f05e 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.GroupScan;
 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 class BinaryTableGroupScan extends MapRDBGroupScan implements DrillHBaseC
                               @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 a1d7f9a..975f1b8 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.GroupScan;
 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 class JsonTableGroupScan extends MapRDBGroupScan {
                             @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/storage-hive/core/pom.xml b/contrib/storage-hive/core/pom.xml
index 0f7ff1f..e0dbb5a 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/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 0000000..50fee9c
--- /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.getStoragePlugin().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 nariveMapRDBScan =
+        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(),
+        nariveMapRDBScan,
+        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 3484ab3..2a2f4fb 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,7 +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;
@@ -40,22 +39,14 @@ import org.apache.drill.exec.store.hive.HiveDrillNativeParquetScan;
 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.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 @@ public class ConvertHiveParquetScanToDrillParquetScan extends StoragePluginOptim
   }
 
   /**
-   * 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
@@ -203,7 +111,7 @@ public class ConvertHiveParquetScanToDrillParquetScan extends StoragePluginOptim
         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) {
@@ -341,18 +249,4 @@ public class ConvertHiveParquetScanToDrillParquetScan extends StoragePluginOptim
 
     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/HiveMetadataProvider.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveMetadataProvider.java
index c877564..6da6c40 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
@@ -375,7 +375,7 @@ public class HiveMetadataProvider {
     }
 
     /**
-     * @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<>();
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 449a6f9..ced8b01 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
@@ -34,9 +34,11 @@ 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.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.store.AbstractStoragePlugin;
 import org.apache.drill.exec.store.SchemaConfig;
 import org.apache.drill.exec.store.StoragePluginOptimizerRule;
@@ -165,14 +167,18 @@ public class HiveStoragePlugin extends AbstractStoragePlugin {
 
   @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();
+    if (options.getBoolean(ExecConstants.HIVE_OPTIMIZE_MAPRDB_JSON_SCAN_WITH_NATIVE_READER)) {
+      ruleBuilder.add(ConvertHiveMapRDBJsonScanToDrillMapRDBJsonScan.INSTANCE);
+    }
+    return ruleBuilder.build();
   }
 
   private static HiveConf createHiveConf(final Map<String, String> hiveConfigOverride) {
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 f5ebc5d..e6178b2 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 @@ public class HiveTableWrapper {
   @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 class HiveTableWrapper {
     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 5279f28..c8efb65 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.Decimal18Holder;
 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.MetaStoreUtils;
 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,18 +73,20 @@ import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
 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;
 
@@ -528,7 +530,7 @@ public class HiveUtilities {
   }
 
   /**
-   * 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 +557,129 @@ public class HiveUtilities {
       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;
+  }
 }
 
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 fd9701c..556deb2 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 @@ public class TestHiveDrillNativeParquetReader extends HiveTestBase {
 
   @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);
   }
 
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 35cc351..634f670 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
@@ -387,9 +387,17 @@ public final class 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 SLICE_TARGET = "planner.slice_target";
   public static final long SLICE_TARGET_DEFAULT = 100000l;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
index e543605..06b4c57 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
@@ -169,6 +169,8 @@ public class SystemOptionManager extends BaseOptionManager implements AutoClosea
       new OptionDefinition(ExecConstants.KAFKA_POLL_TIMEOUT_VALIDATOR),
       new OptionDefinition(ExecConstants.KAFKA_READER_READ_NUMBERS_AS_DOUBLE_VALIDATOR),
       new OptionDefinition(ExecConstants.HIVE_OPTIMIZE_SCAN_WITH_NATIVE_READERS_VALIDATOR),
+      new OptionDefinition(ExecConstants.HIVE_OPTIMIZE_PARQUET_SCAN_WITH_NATIVE_READER_VALIDATOR),
+      new OptionDefinition(ExecConstants.HIVE_OPTIMIZE_MAPRDB_JSON_SCAN_WITH_NATIVE_READER_VALIDATOR),
       new OptionDefinition(ExecConstants.SLICE_TARGET_OPTION),
       new OptionDefinition(ExecConstants.AFFINITY_FACTOR),
       new OptionDefinition(ExecConstants.MAX_WIDTH_GLOBAL),
diff --git a/exec/java-exec/src/main/resources/drill-module.conf b/exec/java-exec/src/main/resources/drill-module.conf
index 239d556..99caeab 100644
--- a/exec/java-exec/src/main/resources/drill-module.conf
+++ b/exec/java-exec/src/main/resources/drill-module.conf
@@ -547,6 +547,8 @@ drill.exec.options: {
     security.admin.users: "%drill_process_user%",
     store.format: "parquet",
     store.hive.optimize_scan_with_native_readers: false,
+    store.hive.parquet.optimize_scan_with_native_reader: false,
+    store.hive.maprdb_json.optimize_scan_with_native_reader: false,
     store.json.all_text_mode: false,
     store.json.writer.allow_nan_inf: true,
     store.json.reader.allow_nan_inf: true,


[drill] 08/09: DRILL-6340 Output Batch Control in Project using the RecordBatchSizer

Posted by vi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 67d992c6bbe11bea2350538db90090ccf8942096
Author: karthik <km...@maprtech.com>
AuthorDate: Wed Mar 21 13:52:03 2018 -0700

    DRILL-6340 Output Batch Control in Project using the RecordBatchSizer
    
    Changes required to implement Output Batch Sizing in Project using the RecordBatchSizer.
    
    closes #1302
---
 .../scanner/persistence/AnnotationDescriptor.java  |   2 +
 .../drill/exec/expr/AbstractExecExprVisitor.java   |  58 ++++
 .../drill/exec/expr/ValueVectorReadExpression.java |   8 +-
 .../exec/expr/ValueVectorWriteExpression.java      |   9 +-
 .../exec/expr/annotations/FunctionTemplate.java    |  34 ++
 .../apache/drill/exec/expr/fn/DrillFuncHolder.java |  29 +-
 .../drill/exec/expr/fn/FunctionAttributes.java     |   8 +
 .../drill/exec/expr/fn/impl/ByteSubstring.java     |   3 +-
 .../exec/expr/fn/impl/CastVarCharVar16Char.java    |   3 +-
 .../drill/exec/expr/fn/impl/CharSubstring.java     |   3 +-
 .../drill/exec/expr/fn/impl/ContextFunctions.java  |  10 +-
 .../drill/exec/expr/fn/impl/DateTypeFunctions.java |   4 +-
 .../exec/expr/fn/impl/SimpleCastFunctions.java     |   3 +-
 .../drill/exec/expr/fn/impl/StringFunctions.java   | 108 ++++---
 .../exec/expr/fn/impl/conv/BigIntBEConvertTo.java  |   4 +-
 .../exec/expr/fn/impl/conv/BigIntConvertTo.java    |   4 +-
 .../expr/fn/impl/conv/BigIntVLongConvertTo.java    |   4 +-
 .../expr/fn/impl/conv/BooleanByteConvertTo.java    |   4 +-
 .../expr/fn/impl/conv/DateEpochBEConvertTo.java    |   4 +-
 .../exec/expr/fn/impl/conv/DateEpochConvertTo.java |   4 +-
 .../exec/expr/fn/impl/conv/DoubleBEConvertTo.java  |   4 +-
 .../exec/expr/fn/impl/conv/DoubleConvertTo.java    |   4 +-
 .../exec/expr/fn/impl/conv/DummyConvertFrom.java   |   3 +-
 .../exec/expr/fn/impl/conv/DummyConvertTo.java     |   3 +-
 .../exec/expr/fn/impl/conv/FloatBEConvertTo.java   |   4 +-
 .../exec/expr/fn/impl/conv/FloatConvertTo.java     |   4 +-
 .../exec/expr/fn/impl/conv/IntBEConvertTo.java     |   4 +-
 .../drill/exec/expr/fn/impl/conv/IntConvertTo.java |   4 +-
 .../exec/expr/fn/impl/conv/IntVIntConvertTo.java   |   4 +-
 .../exec/expr/fn/impl/conv/JsonConvertTo.java      |   6 +-
 .../expr/fn/impl/conv/SmallIntBEConvertTo.java     |   4 +-
 .../exec/expr/fn/impl/conv/SmallIntConvertTo.java  |   4 +-
 .../expr/fn/impl/conv/TimeEpochBEConvertTo.java    |   4 +-
 .../exec/expr/fn/impl/conv/TimeEpochConvertTo.java |   4 +-
 .../fn/impl/conv/TimeStampEpochBEConvertTo.java    |   4 +-
 .../exec/expr/fn/impl/conv/TinyIntConvertTo.java   |   4 +-
 .../exec/expr/fn/impl/conv/UInt4BEConvertTo.java   |   4 +-
 .../exec/expr/fn/impl/conv/UInt4ConvertTo.java     |   4 +-
 .../exec/expr/fn/impl/conv/UInt8ConvertTo.java     |   4 +-
 .../exec/expr/fn/impl/conv/UTF16ConvertFrom.java   |   3 +-
 .../exec/expr/fn/impl/conv/UTF16ConvertTo.java     |   4 +-
 .../exec/expr/fn/impl/conv/UTF8ConvertFrom.java    |   3 +-
 .../exec/expr/fn/impl/conv/UTF8ConvertTo.java      |   3 +-
 .../fn/output/OutputWidthCalculator.java}          |  22 +-
 .../expr/fn/output/OutputWidthCalculators.java     | 113 +++++++
 .../impl/project/OutputSizeEstimateConstants.java  |  46 +++
 .../impl/project/OutputWidthExpression.java        | 147 +++++++++
 .../physical/impl/project/OutputWidthVisitor.java  | 278 ++++++++++++++++
 ...Projector.java => OutputWidthVisitorState.java} |  26 +-
 .../impl/project/ProjectMemoryManager.java         | 310 ++++++++++++++++++
 .../physical/impl/project/ProjectRecordBatch.java  |  84 ++++-
 .../exec/physical/impl/project/Projector.java      |   6 +-
 .../physical/impl/project/ProjectorTemplate.java   |  15 +-
 .../apache/drill/exec/record/RecordBatchSizer.java |  83 +++--
 .../org/apache/drill/exec/record/TypedFieldId.java |  32 ++
 .../exec/compile/TestLargeFileCompilation.java     |   2 +
 .../exec/physical/unit/TestOutputBatchSize.java    | 350 +++++++++++++++++++++
 57 files changed, 1744 insertions(+), 165 deletions(-)

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 4669229..67df91e 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 final class AnnotationDescriptor {
           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/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 0000000..36e2bba
--- /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/ValueVectorReadExpression.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorReadExpression.java
index 410c48a..370960b 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.expression.visitors.ExprVisitor;
 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 class ValueVectorReadExpression implements LogicalExpression {
 
   @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 e2a947b..f419c88 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.TypeProtos.MajorType;
 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 class ValueVectorWriteExpression implements LogicalExpression {
 
   @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 1a5470b..11914ea 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.FunctionAttributes;
 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 @@ public @interface FunctionTemplate {
   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/DrillFuncHolder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFuncHolder.java
index 240ff27..b169fb1 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,9 +17,12 @@
  */
 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;
@@ -38,18 +41,15 @@ import org.apache.drill.exec.expr.ClassGenerator.HoldingContainer;
 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 {
 
@@ -106,6 +106,7 @@ public abstract class DrillFuncHolder extends AbstractFuncHolder {
     return attributes.isNiladic();
   }
 
+
   /**
    * Generates string representation of function input parameters:
    * PARAMETER_TYPE_1-PARAMETER_MODE_1,PARAMETER_TYPE_2-PARAMETER_MODE_2
@@ -291,6 +292,14 @@ public abstract class DrillFuncHolder extends AbstractFuncHolder {
     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();
   }
@@ -337,6 +346,4 @@ public abstract class DrillFuncHolder extends AbstractFuncHolder {
         + ", 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 4fd5be7..6d1b767 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 class FunctionAttributes {
     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/impl/ByteSubstring.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/ByteSubstring.java
index bcbfeb5..e1d0258 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 @@ import org.apache.drill.exec.expr.holders.VarBinaryHolder;
  */
 @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 edb9291..1a929af 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.Var16CharHolder;
 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 ee6baff..5c47692 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 @@ import org.apache.drill.exec.expr.holders.VarCharHolder;
  */
 @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 e3241f2..590ab7d 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.Output;
 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 @@ public class ContextFunctions {
   /**
    * 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 class ContextFunctions {
   /**
    * 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 class ContextFunctions {
   /**
    * 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 07accf1..afd5f5d 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.TimeHolder;
 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 class DateTypeFunctions {
 
     }
 
-    @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/SimpleCastFunctions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/SimpleCastFunctions.java
index 807fbb9..196445e 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 class SimpleCastFunctions {
   @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/StringFunctions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java
index ad3f379..4309848 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 @@ package org.apache.drill.exec.expr.fn.impl;
 
 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.IntHolder;
 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 class StringFunctions{
   /*
    * 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 class StringFunctions{
   }
 
 
-  @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 class StringFunctions{
   @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 class StringFunctions{
   @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 class StringFunctions{
   // 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 class StringFunctions{
     }
   }
 
-  @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 class StringFunctions{
     }
   }
 
-  @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 class StringFunctions{
     }
   }
 
-  @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 class StringFunctions{
   // 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 class StringFunctions{
   }
 
   //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 class StringFunctions{
   @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 class StringFunctions{
   }
 
   //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 class StringFunctions{
   @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 class StringFunctions{
   @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 class StringFunctions{
   @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 class StringFunctions{
   @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 class StringFunctions{
   /**
    * 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 class StringFunctions{
   /**
    * 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 class StringFunctions{
   /**
    * 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 class StringFunctions{
   /**
    * 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 class StringFunctions{
   /**
    * 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 class StringFunctions{
   /**
    * 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 class StringFunctions{
     } // 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 class StringFunctions{
   @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 class StringFunctions{
   @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 class StringFunctions{
   @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 class StringFunctions{
   @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 class StringFunctions{
   @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 class StringFunctions{
 
   // 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 class StringFunctions{
 
   // 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 class StringFunctions{
   /**
   * 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 class StringFunctions{
   /**
   * 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 class StringFunctions{
   /**
   * 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 class StringFunctions{
   @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 eacf535..b8bd968 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.Output;
 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 8183479..35e659b 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.Output;
 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 26c2033..b694197 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.Output;
 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 c7a96e8..046258c 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.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.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 69aab45..e6c88e9 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.Output;
 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 990ebea..74ec326 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.Output;
 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 e0b5cb1..a2718e7 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.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 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 cd183d8..225d3b9 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.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 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 aac6615..50e4cf0 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 @@ import org.apache.drill.exec.expr.holders.VarBinaryHolder;
  * 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 ba350c5..a17dbe8 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 @@ import org.apache.drill.exec.expr.holders.VarBinaryHolder;
  * 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 0589d8d..e03b173 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.Output;
 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 b624471..25f1272 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.Output;
 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 2f1751a..b33c159 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.Output;
 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 a32ab4d..a9f7d8f 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.Output;
 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 bc0347d..ed0a269 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.Output;
 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 68fdb28..e384892 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 @@ public class JsonConvertTo {
 
   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 class JsonConvertTo {
     }
   }
 
-  @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 420fe34..0b14313 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.Output;
 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 c228d96..bee0c64 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.Output;
 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 8effc89..a7a08f2 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.Output;
 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 39fe3b6..c17efbf 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.Output;
 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 a34c6f8..98a4296 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.Output;
 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 fa7ccb0..3ca23db 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.Output;
 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 c18d281..bda7175 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.Output;
 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 ea574c2..b3777a8 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.Output;
 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 446fe72..46f3884 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.Output;
 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 baad4f8..298213e 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.annotations.Param;
 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 71dddb3..0cccb94 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 @@ package org.apache.drill.exec.expr.fn.impl.conv;
 
 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.annotations.Param;
 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 08de915..68a7320 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.annotations.Param;
 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 1eef114..fb7ad63 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.annotations.Param;
 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/physical/impl/project/Projector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/output/OutputWidthCalculator.java
similarity index 52%
copy from exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/Projector.java
copy to exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/output/OutputWidthCalculator.java
index 455d643..6eadbf5 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/expr/fn/output/OutputWidthCalculator.java
@@ -15,21 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.physical.impl.project;
 
-import java.util.List;
+package org.apache.drill.exec.expr.fn.output;
 
-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 org.apache.drill.exec.physical.impl.project.OutputWidthExpression.FixedLenExpr;
 
-public interface Projector {
+import java.util.List;
 
-  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);
+/**
+ * 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 {
 
-  public static TemplateClassDefinition<Projector> TEMPLATE_DEFINITION = new TemplateClassDefinition<Projector>(Projector.class, ProjectorTemplate.class);
+  int getOutputWidth(List<FixedLenExpr> args);
 
-}
\ No newline at end of file
+}
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 0000000..7c0bb38
--- /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/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 0000000..1c65508
--- /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 0000000..b9240d6
--- /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 0000000..cb58795
--- /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/Projector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/OutputWidthVisitorState.java
similarity index 53%
copy from exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/Projector.java
copy to exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/OutputWidthVisitorState.java
index 455d643..c0e0cb1 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/OutputWidthVisitorState.java
@@ -17,19 +17,21 @@
  */
 package org.apache.drill.exec.physical.impl.project;
 
-import java.util.List;
+public class OutputWidthVisitorState {
 
-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;
+    ProjectMemoryManager manager;
+    ProjectMemoryManager.OutputColumnType outputColumnType;
 
-public interface Projector {
+    public OutputWidthVisitorState(ProjectMemoryManager manager, ProjectMemoryManager.OutputColumnType outputColumnType) {
+        this.manager = manager;
+        this.outputColumnType = outputColumnType;
+    }
 
-  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 ProjectMemoryManager getManager() {
+        return manager;
+    }
 
-  public static TemplateClassDefinition<Projector> TEMPLATE_DEFINITION = new TemplateClassDefinition<Projector>(Projector.class, ProjectorTemplate.class);
-
-}
\ No newline at end of file
+    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 0000000..f461b09
--- /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 8a88db9..4bc63c0 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.expression.fn.CastFunctions;
 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.AbstractSingleRecordBatch;
 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 @@ public class ProjectRecordBatch extends AbstractSingleRecordBatch<Project> {
   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 @@ public class ProjectRecordBatch extends AbstractSingleRecordBatch<Project> {
 
   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 @@ public class ProjectRecordBatch extends AbstractSingleRecordBatch<Project> {
 
     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 @@ public class ProjectRecordBatch extends AbstractSingleRecordBatch<Project> {
             // 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 @@ public class ProjectRecordBatch extends AbstractSingleRecordBatch<Project> {
             }
           }
           incomingRecordCount = incoming.getRecordCount();
+          memoryManager.update();
+          logger.trace("doWork():[1] memMgr RC {}, incoming rc {},  incoming {}, Project {}",
+                       memoryManager.getOutputRowCount(), incomingRecordCount, incoming, this);
         }
       }
     }
@@ -206,12 +223,21 @@ public class ProjectRecordBatch extends AbstractSingleRecordBatch<Project> {
     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 @@ public class ProjectRecordBatch extends AbstractSingleRecordBatch<Project> {
       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 @@ public class ProjectRecordBatch extends AbstractSingleRecordBatch<Project> {
 
   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 @@ public class ProjectRecordBatch extends AbstractSingleRecordBatch<Project> {
     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 @@ public class ProjectRecordBatch extends AbstractSingleRecordBatch<Project> {
   }
 
   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 @@ public class ProjectRecordBatch extends AbstractSingleRecordBatch<Project> {
   }
 
   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 @@ public class ProjectRecordBatch extends AbstractSingleRecordBatch<Project> {
     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 @@ public class ProjectRecordBatch extends AbstractSingleRecordBatch<Project> {
     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 @@ public class ProjectRecordBatch extends AbstractSingleRecordBatch<Project> {
 
     for (NamedExpression namedExpression : exprs) {
       result.clear();
-
       if (classify && namedExpression.getExpr() instanceof SchemaPath) {
         classifyExpr(namedExpression, incomingBatch, result);
 
@@ -385,6 +431,7 @@ public class ProjectRecordBatch extends AbstractSingleRecordBatch<Project> {
               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 @@ public class ProjectRecordBatch extends AbstractSingleRecordBatch<Project> {
               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 @@ public class ProjectRecordBatch extends AbstractSingleRecordBatch<Project> {
         // 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 @@ public class ProjectRecordBatch extends AbstractSingleRecordBatch<Project> {
           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 @@ public class ProjectRecordBatch extends AbstractSingleRecordBatch<Project> {
 
         // 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 @@ public class ProjectRecordBatch extends AbstractSingleRecordBatch<Project> {
         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 @@ public class ProjectRecordBatch extends AbstractSingleRecordBatch<Project> {
       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 455d643..5e5e7eb 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 499e55b..ec40967 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.TransferPair;
 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 abstract class ProjectorTemplate implements Projector {
   }
 
   @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 abstract class ProjectorTemplate implements Projector {
           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/record/RecordBatchSizer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchSizer.java
index a5cb05b..4b8ae80 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchSizer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchSizer.java
@@ -21,6 +21,7 @@ import java.util.Set;
 import java.util.Map;
 
 import org.apache.drill.common.map.CaseInsensitiveMap;
+import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.expr.TypeHelper;
@@ -42,6 +43,8 @@ import org.apache.drill.exec.vector.VariableWidthVector;
 
 import com.google.common.collect.Sets;
 import org.apache.drill.exec.vector.complex.RepeatedVariableWidthVectorLike;
+import org.bouncycastle.util.Strings;
+
 import static org.apache.drill.exec.vector.AllocationHelper.STD_REPETITION_FACTOR;
 
 /**
@@ -202,30 +205,7 @@ public class RecordBatchSizer {
      * For repeated column, we assume repetition of 10.
      */
     public int getStdNetSizePerEntry() {
-      int stdNetSize;
-      try {
-        stdNetSize = TypeHelper.getSize(metadata.getType());
-      } catch (Exception e) {
-        stdNetSize = 0;
-      }
-
-      if (isOptional) {
-        stdNetSize += BIT_VECTOR_WIDTH;
-      }
-
-      if (isRepeated) {
-        stdNetSize = (stdNetSize * STD_REPETITION_FACTOR) + OFFSET_VECTOR_WIDTH;
-      }
-
-      for (ColumnSize columnSize : children.values()) {
-        stdNetSize += columnSize.getStdNetSizePerEntry();
-      }
-
-      if (isRepeatedList()) {
-        stdNetSize = (stdNetSize * STD_REPETITION_FACTOR) + OFFSET_VECTOR_WIDTH;
-      }
-
-      return stdNetSize;
+      return getStdNetSizePerEntryCommon(metadata.getType(), isOptional, isRepeated, isRepeatedList(), children);
     }
 
     /**
@@ -598,8 +578,61 @@ public class RecordBatchSizer {
 
   }
 
+   public static int getStdNetSizePerEntryCommon(TypeProtos.MajorType majorType, boolean isOptional, boolean isRepeated,
+                                                 boolean isRepeatedList, Map<String, ColumnSize> children) {
+    int stdNetSize;
+    try {
+      stdNetSize = TypeHelper.getSize(majorType);
+    } catch (Exception e) {
+      stdNetSize = 0;
+    }
+
+    if (isOptional) {
+      stdNetSize += BIT_VECTOR_WIDTH;
+    }
+
+    if (isRepeated) {
+      stdNetSize = (stdNetSize * STD_REPETITION_FACTOR) + OFFSET_VECTOR_WIDTH;
+    }
+
+    if (children != null) {
+      for (ColumnSize columnSize : children.values()) {
+        stdNetSize += columnSize.getStdNetSizePerEntry();
+      }
+    }
+
+    if (isRepeatedList) {
+      stdNetSize = (stdNetSize * STD_REPETITION_FACTOR) + OFFSET_VECTOR_WIDTH;
+    }
+
+    return stdNetSize;
+  }
+
+  private ColumnSize getComplexColumn(String path) {
+    String[] segments = Strings.split(path, '.');
+    Map<String, ColumnSize> map = columnSizes;
+    return getComplexColumnImpl(segments, 0, map);
+  }
+
+  private ColumnSize getComplexColumnImpl(String[] segments, int level, Map<String, ColumnSize> map) {
+    ColumnSize result = map.get(segments[level]);
+    if (result == null || level == segments.length - 1) {
+      return result;
+    }
+    map = result.getChildren();
+    if (map == null) {
+      return null;
+    }
+    return getComplexColumnImpl(segments, level + 1, map);
+  }
+
   public ColumnSize getColumn(String name) {
-    return columnSizes.get(name);
+    final RecordBatchSizer.ColumnSize columnSize =  columnSizes.get(name);
+    if (columnSize != null) {
+      return columnSize;
+    } else {
+      return getComplexColumn(name);
+    }
   }
 
   // This keeps information for only top level columns. Information for nested
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/TypedFieldId.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/TypedFieldId.java
index 9a4e0da..a0c4676 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/TypedFieldId.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/TypedFieldId.java
@@ -283,4 +283,36 @@ public class TypedFieldId {
         + ", remainder=" + remainder + "]";
   }
 
+  /**
+   * Generates the full path to a field from the typefield ids
+   *
+   * @param typeFieldId
+   * @param recordBatch
+   * @return
+   */
+  public static String getPath(TypedFieldId typeFieldId, RecordBatch recordBatch) {
+    StringBuilder name = new StringBuilder();
+    final String SEPARATOR = ".";
+    final int[] fieldIds = typeFieldId.getFieldIds();
+    VectorWrapper<?> topLevel = recordBatch.getValueAccessorById(null, fieldIds[0]);
+    name.append(topLevel.getField().getName());
+    // getChildWrapper(int[] fieldIds) is used to walk down the list of fieldIds.
+    // getChildWrapper() has a quirk where if the fieldIds array is of length == 1
+    // then it would just return 'this'.
+    // For example, if you had a field 'a.b' with field ids {1, 2} and you had the
+    // VectorWrapper for 'a', say 'aVW'. Then calling aVW.getChildWrapper({2}) returns
+    // aVW and not the vectorWrapper for 'b'.
+    // The code works around this quirk by always querying 2 levels deep.
+    // i.e. childVectorWrapper = parentVectorWrapper.gerChildWrapper({parentFieldId, childFieldId})
+    int[] lookupLevel = new int[2];
+    for (int i = 0; i < fieldIds.length - 1; i++) {
+      lookupLevel[0] = fieldIds[i];
+      // this is the level for which the actual lookup is done
+      lookupLevel[1] = fieldIds[i + 1];
+      topLevel = topLevel.getChildWrapper(lookupLevel);
+      name.append(SEPARATOR + topLevel.getField().getName());
+    }
+    return name.toString();
+  }
+
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestLargeFileCompilation.java b/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestLargeFileCompilation.java
index dd90edd..07dbae2 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestLargeFileCompilation.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestLargeFileCompilation.java
@@ -145,12 +145,14 @@ public class TestLargeFileCompilation extends BaseTestQuery {
     testNoResult(ITERATION_COUNT, LARGE_QUERY_GROUP_BY);
   }
 
+  @Ignore // TODO: DRILL-6529
   @Test
   public void testEXTERNAL_SORT() throws Exception {
     testNoResult("alter session set `%s`='JDK'", ClassCompilerSelector.JAVA_COMPILER_OPTION);
     testNoResult(ITERATION_COUNT, LARGE_QUERY_ORDER_BY);
   }
 
+  @Ignore // TODO: DRILL-6529
   @Test
   public void testTOP_N_SORT() throws Exception {
     testNoResult("alter session set `%s`='JDK'", ClassCompilerSelector.JAVA_COMPILER_OPTION);
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestOutputBatchSize.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestOutputBatchSize.java
index da83b00..a029832 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestOutputBatchSize.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestOutputBatchSize.java
@@ -19,6 +19,7 @@ package org.apache.drill.exec.physical.unit;
 
 import com.google.common.collect.Lists;
 import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.directory.api.util.Strings;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.expression.SchemaPath;
 
@@ -27,6 +28,7 @@ import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.config.FlattenPOP;
 import org.apache.drill.exec.physical.config.HashJoinPOP;
 import org.apache.drill.exec.physical.config.MergeJoinPOP;
+import org.apache.drill.exec.physical.config.Project;
 import org.apache.drill.exec.physical.config.UnionAll;
 import org.apache.drill.exec.physical.impl.ScanBatch;
 import org.apache.drill.exec.record.RecordBatchSizer;
@@ -77,6 +79,354 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
   }
 
   @Test
+  public void testProjectMap() throws Exception {
+    // create input rows like this.
+    // "a" : 5, "b" : wideString, "c" : [{"trans_id":"t1", amount:100, trans_time:7777777, type:sports},
+    //                                   {"trans_id":"t1", amount:100, trans_time:8888888, type:groceries}]
+    StringBuilder batchString = new StringBuilder("[");
+    for (int i = 0; i < numRows; i++) {
+      batchString.append("{\"a\": 5, " + "\"b\" : " + "\"" + "abc" + "\"," +
+                         " \"c\" : { \"trans_id\":\"t1\", \"amount\":100, \"trans_time\":7777777, \"type\":\"sports\"}," +
+                         " \"d\": { \"trans_id\":\"t2\", \"amount\":1000, \"trans_time\":8888888, \"type\":\"groceries\"}");
+      batchString.append(i != numRows - 1 ? "}," : "}]");
+    }
+    List<String> inputJsonBatches = Lists.newArrayList();
+    inputJsonBatches.add(batchString.toString());
+
+    StringBuilder expectedString = new StringBuilder("[");
+    for (int i = 0; i < numRows; i++) {
+      expectedString.append("{\"aplusamount\": 105");
+      expectedString.append(i != numRows - 1 ? "}," : "}]");
+    }
+
+    List<String> expectedJsonBatches = Lists.newArrayList();
+    expectedJsonBatches.add(expectedString.toString());
+
+    String[] baselineColumns = new String[1];
+    baselineColumns[0] = "aplusamount";
+
+    String[] expr = {"a + c.amount ", baselineColumns[0]};
+
+    Project projectConf = new Project(parseExprs(expr), null);
+    mockOpContext(projectConf, initReservation, maxAllocation);
+
+    long totalSize = getExpectedSize(expectedJsonBatches);
+
+    fragContext.getOptions().setLocalOption("drill.exec.memory.operator.output_batch_size", totalSize / 2);
+
+    OperatorTestBuilder opTestBuilder = opTestBuilder()
+            .physicalOperator(projectConf)
+            .inputDataStreamJson(inputJsonBatches)
+            .baselineColumns(baselineColumns)
+            .expectedNumBatches(2)  // verify number of batches
+            .expectedBatchSize(totalSize / 2); // verify batch size.
+
+    Long[] baseLineValues = {(5l + 100l)}; // a + c.amount
+    for (int i = 0; i < numRows; i++) {
+      opTestBuilder.baselineValues(baseLineValues);
+    }
+    opTestBuilder.go();
+  }
+
+  @Test
+  public void testProjectVariableWidthFunctions() throws  Exception {
+    //size calculators
+    StringBuilder batchString = new StringBuilder("[");
+    String strValue = "abcde";
+    for (int i = 0; i < numRows; i++) {
+      batchString.append("{\"a\" : " + "\"" + strValue + "\"");
+      batchString.append(i != numRows - 1 ? "}," : "}]");
+    }
+    List<String> inputJsonBatches = Lists.newArrayList();
+    inputJsonBatches.add(batchString.toString());
+
+    // inputSize, as calculated below will be numRows * (inputRowsize),
+    // inputRowSize = metadata cols + sizeof("abcde"), numRows = 4000
+    // So, inputSize = 4000 * ( 4 + 1 + 5 ) = 40000
+    // inputSize is used as the batch memory limit for the tests.
+    // Depending on the function being evaluated, different output batch counts will be expected
+    long inputSize = getExpectedSize(inputJsonBatches);
+    String inputSizeStr = inputSize + "";
+
+    String [][] functions =
+                         {  //{ OP name, OP result, OP SQL str, Memory Limit, Num Expected Batches }
+
+                         // concat() o/p size will be 2 x input size, so at least 2 batches expected
+                         {"concat", strValue + strValue, "concat(a,a)", inputSizeStr, 2 + ""},
+                         // upper() o/p size will same as input size, so at least 1 batch is expected
+                         {"upper", strValue.toUpperCase(),"upper(a)", inputSizeStr, 1 + ""},
+                         // repeat() is assumed to produce a row-size of 50.
+                         // input row size is 10 (null vector + offset vector + abcde)
+                         // so at least 5 batches are expected
+                         {"repeat", strValue + strValue, "repeatstr(a, 2)", inputSizeStr, 5 + ""},
+                         // substr() is assumed to produce a row size which is same as input
+                         // so at least 1 batch is expected
+                         {"substr", strValue.substring(0, 4), "substr(a, 1, 4)", inputSizeStr, 1 + "" }
+                      };
+
+    for (String[] fn : functions) {
+      String outputColumnName = fn[0] + "_result";
+      String operationResult = fn[1];
+      String exprStr = fn[2];
+      long memoryLimit = Long.valueOf(fn[3]);
+      int expectedNumBatches = Integer.valueOf(fn[4]);
+
+      StringBuilder expectedString = new StringBuilder("[");
+      for (int i = 0; i < numRows; i++) {
+        expectedString.append("{\"" + outputColumnName + "\":" + operationResult);
+        expectedString.append(i != numRows - 1 ? "}," : "}]");
+      }
+
+      List<String> expectedJsonBatches = Lists.newArrayList();
+      expectedJsonBatches.add(expectedString.toString());
+
+      String[] baselineColumns = new String[1];
+      baselineColumns[0] = outputColumnName;
+
+      String[] expr = {exprStr, baselineColumns[0]};
+
+      Project projectConf = new Project(parseExprs(expr), null);
+      mockOpContext(projectConf, initReservation, maxAllocation);
+
+      fragContext.getOptions().setLocalOption("drill.exec.memory.operator.output_batch_size", memoryLimit);
+
+      OperatorTestBuilder opTestBuilder = opTestBuilder()
+              .physicalOperator(projectConf)
+              .inputDataStreamJson(inputJsonBatches)
+              .baselineColumns(baselineColumns)
+              .expectedNumBatches(expectedNumBatches)  // verify number of batches
+              .expectedBatchSize(memoryLimit); // verify batch size.
+
+      String[] baseLineValues = {operationResult}; //operation(a, a)
+      for (int i = 0; i < numRows; i++) {
+        opTestBuilder.baselineValues(baseLineValues);
+      }
+      opTestBuilder.go();
+    }
+  }
+
+
+  @Test
+  public void testProjectFixedWidthTransfer() throws Exception {
+    testProjectFixedWidthImpl(true, 100);
+  }
+
+  @Test
+  public void testProjectFixedWidthNewColumn() throws Exception {
+    testProjectFixedWidthImpl(false, 100);
+  }
+
+   /**
+    * Tests BatchSizing of fixed-width transfers and new column creations in Project.
+    * Transfer: Evaluates 'select *'
+    * New Columns: Evalutes 'select C0 + 5 as C0 ... C[columnCount] + 5 as C[columnCount]
+    * @param transfer
+    * @throws Exception
+    */
+
+  public void testProjectFixedWidthImpl(boolean transfer, int columnCount) throws  Exception {
+
+    //generate a row with N columns C0..C[columnCount], value in a column is same as column id
+    StringBuilder jsonRow = new StringBuilder("{");
+    String[] baselineColumns = new String [columnCount];
+    Object[] baselineValues = new Long[columnCount];
+
+    int exprSize = (transfer ? 2 : 2 * columnCount);
+    String[] expr = new String[exprSize];
+
+    // Expr for a 'select *' as expected by parseExprs()
+    if (transfer) {
+      expr[0] = "`**`";
+      expr[1] = "`**`";
+    }
+
+    for (int i = 0; i < columnCount; i++) {
+      jsonRow.append("\"" + "C" + i + "\": " + i + ((i == columnCount - 1) ? "" : ","));
+      baselineColumns[i] = "C" + i;
+      if (!transfer) {
+        expr[i * 2] = baselineColumns[i] + " + 5";
+        expr[i * 2 + 1] = baselineColumns[i];
+      }
+      baselineValues[i] = (long)(transfer ? i : i + 5);
+    }
+    jsonRow.append("}");
+    StringBuilder batchString = new StringBuilder("[");
+    for (int i = 0; i < numRows; i++) {
+      batchString.append(jsonRow + ((i == numRows - 1) ? "" : ","));
+    }
+    batchString.append("]");
+    List<String> inputJsonBatches = Lists.newArrayList();
+    inputJsonBatches.add(batchString.toString());
+
+    List<String> expectedJsonBatches = Lists.newArrayList();
+    expectedJsonBatches.add(batchString.toString());
+
+    Project projectConf = new Project(parseExprs(expr), null);
+    mockOpContext(projectConf, initReservation, maxAllocation);
+
+    long totalSize = getExpectedSize(expectedJsonBatches);
+
+    fragContext.getOptions().setLocalOption("drill.exec.memory.operator.output_batch_size", totalSize / 2);
+
+
+    OperatorTestBuilder opTestBuilder = opTestBuilder()
+            .physicalOperator(projectConf)
+            .inputDataStreamJson(inputJsonBatches)
+            .baselineColumns(baselineColumns)
+            .expectedNumBatches(2)  // verify number of batches
+            .expectedBatchSize(totalSize / 2); // verify batch size.
+
+    for (int i = 0; i < numRows; i++) {
+      opTestBuilder.baselineValues(baselineValues);
+    }
+    opTestBuilder.go();
+  }
+
+  @Test
+  public void testProjectVariableWidthTransfer() throws Exception {
+    testProjectVariableWidthImpl(true, 50, "ABCDEFGHIJ");
+  }
+
+  @Test
+  public void testProjectVariableWidthNewColumn() throws Exception {
+    testProjectVariableWidthImpl(false, 50, "ABCDEFGHIJ");
+  }
+
+  @Test
+  public void testProjectZeroWidth() throws Exception {
+    testProjectVariableWidthImpl(true, 50, "");
+  }
+
+
+  public void testProjectVariableWidthImpl(boolean transfer, int columnCount, String testString) throws Exception {
+
+    StringBuilder jsonRow = new StringBuilder("{");
+    String[] baselineColumns = new String [columnCount];
+    Object[] baselineValues = new String[columnCount];
+    int exprSize = (transfer ? 2 : 2 * columnCount);
+    String[] expr = new String[exprSize];
+
+    // Expr for a 'select *' as expected by parseExprs()
+    if (transfer) {
+      expr[0] = "`**`";
+      expr[1] = "`**`";
+    }
+
+    for (int i = 0; i < columnCount; i++) {
+      jsonRow.append("\"" + "C" + i + "\": " + "\"" + testString + "\"" + ((i == columnCount - 1) ? "" : ","));
+      baselineColumns[i] = "C" + i;
+      if (!transfer) {
+        expr[i * 2] = "lower(" + baselineColumns[i] + ")";
+        expr[i * 2 + 1] = baselineColumns[i];
+      }
+      baselineValues[i] = (transfer ? testString : Strings.lowerCase(testString));
+    }
+    jsonRow.append("}");
+    StringBuilder batchString = new StringBuilder("[");
+    for (int i = 0; i < numRows; i++) {
+      batchString.append(jsonRow + ((i == numRows - 1) ? "" : ","));
+    }
+    batchString.append("]");
+    List<String> inputJsonBatches = Lists.newArrayList();
+    inputJsonBatches.add(batchString.toString());
+
+    List<String> expectedJsonBatches = Lists.newArrayList();
+    expectedJsonBatches.add(batchString.toString());
+
+    Project projectConf = new Project(parseExprs(expr), null);
+    mockOpContext(projectConf, initReservation, maxAllocation);
+
+    long totalSize = getExpectedSize(expectedJsonBatches);
+
+    fragContext.getOptions().setLocalOption("drill.exec.memory.operator.output_batch_size", totalSize / 2);
+
+    OperatorTestBuilder opTestBuilder = opTestBuilder()
+            .physicalOperator(projectConf)
+            .inputDataStreamJson(inputJsonBatches)
+            .baselineColumns(baselineColumns)
+            .expectedNumBatches(2)  // verify number of batches
+            .expectedBatchSize(totalSize / 2); // verify batch size.
+
+    for (int i = 0; i < numRows; i++) {
+      opTestBuilder.baselineValues(baselineValues);
+    }
+    opTestBuilder.go();
+  }
+
+  /**
+   * Test expression with transfer and new columns
+   * @throws Exception
+   */
+  @Test
+  public void testProjectVariableWidthMixed() throws Exception {
+    String testString = "ABCDEFGHIJ";
+    StringBuilder jsonRow = new StringBuilder("{");
+    // 50 new columns and 1 transfer
+    final int colCount = 50 + 1;
+    String[] baselineColumns = new String [colCount];
+    Object[] baselineValues = new String[colCount];
+    int exprSize = 2 * colCount;
+    String[] expr = new String[exprSize];
+
+    // columns C1 ... C50
+    for (int i = 1; i < colCount; i++) {
+      jsonRow.append("\"" + "C" + i + "\": " + "\"" + testString + "\"" + ((i == colCount - 1) ? "" : ","));
+      baselineColumns[i] = "C" + i;
+      // New columns lower(C1) as C1, ... lower(C50) as C50
+      expr[i * 2] = "lower(" + baselineColumns[i] + ")";
+      expr[i * 2 + 1] = baselineColumns[i];
+
+      baselineValues[i] = Strings.lowerCase(testString);
+    }
+
+
+    //Transfer: C1 as COL1TR
+    expr[0] = "C1";
+    expr[1] = "COL1TR";
+    baselineColumns[0] = "COL1TR";
+    baselineValues[0] = testString;
+    String expectedJsonRow = jsonRow.toString() + ", \"COL1TR\": \"" + testString + "\"}";
+    jsonRow.append("}");
+
+    StringBuilder batchString = new StringBuilder("[");
+    StringBuilder expectedString = new StringBuilder("[");
+
+    for (int i = 0; i < numRows; i++) {
+      batchString.append(jsonRow + ((i == numRows - 1) ? "" : ","));
+      expectedString.append(expectedJsonRow + ((i == numRows - 1) ? "" : ","));
+    }
+    batchString.append("]");
+    expectedString.append("]");
+
+    List<String> inputJsonBatches = Lists.newArrayList();
+    inputJsonBatches.add(batchString.toString());
+
+    List<String> expectedJsonBatches = Lists.newArrayList();
+    expectedJsonBatches.add(expectedString.toString());
+
+    Project projectConf = new Project(parseExprs(expr), null);
+    mockOpContext(projectConf, initReservation, maxAllocation);
+
+    long totalSize = getExpectedSize(expectedJsonBatches);
+
+    fragContext.getOptions().setLocalOption("drill.exec.memory.operator.output_batch_size", totalSize / 2);
+
+    OperatorTestBuilder opTestBuilder = opTestBuilder()
+            .physicalOperator(projectConf)
+            .inputDataStreamJson(inputJsonBatches)
+            .baselineColumns(baselineColumns)
+            .expectedNumBatches(2)  // verify number of batches
+            .expectedBatchSize(totalSize / 2); // verify batch size.
+
+    for (int i = 0; i < numRows; i++) {
+      opTestBuilder.baselineValues(baselineValues);
+    }
+    opTestBuilder.go();
+  }
+
+
+
+  @Test
   public void testFlattenFixedWidth() throws Exception {
     PhysicalOperator flatten = new FlattenPOP(null, SchemaPath.getSimplePath("c"));
     mockOpContext(flatten, initReservation, maxAllocation);


[drill] 05/09: DRILL-6513: Max query memory per node set to Drillbit's maximum direct memory

Posted by vi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 6823a8f241a58a4ae1dc1c6e84ba6a9920cc1643
Author: Salim Achouche <sa...@gmail.com>
AuthorDate: Tue Jun 19 13:10:06 2018 -0700

    DRILL-6513: Max query memory per node set to Drillbit's maximum direct memory
    
    closes #1329
---
 exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)

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 776c469..35cc351 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;
@@ -440,7 +441,7 @@ public final class 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


[drill] 07/09: DRILL-6486: BitVector split and transfer does not work correctly for non byte-multiple transfer lengths

Posted by vi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 947e6e561414182f2060d905daae197f3dd71f9f
Author: karthik <km...@maprtech.com>
AuthorDate: Thu Jun 7 13:23:04 2018 -0700

    DRILL-6486: BitVector split and transfer does not work correctly for non byte-multiple transfer lengths
    
    Fix for the bug in BitVector splitAndTransfer. The logic for handling copy of last-n bits was incorrect for none byte-multiple transfer lengths.
    
    closes #1316
---
 .../drill/exec/record/vector/TestValueVector.java  |   3 +
 .../drill/exec/vector/TestSplitAndTransfer.java    | 108 ++++++++++++++++++++-
 .../org/apache/drill/exec/vector/BitVector.java    |  45 ++++++---
 3 files changed, 141 insertions(+), 15 deletions(-)

diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java b/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
index 52a1585..64b8cae 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
@@ -548,6 +548,7 @@ the interface to load has changed
       m.set(1, 0);
       m.set(100, 0);
       m.set(1022, 1);
+      m.setValueCount(1023);
 
       final BitVector.Accessor accessor = vector.getAccessor();
       assertEquals(1, accessor.get(0));
@@ -560,12 +561,14 @@ the interface to load has changed
       m.set(0, 1);
       m.set(1, 0);
       m.set(1, 0);
+      m.setValueCount(2);
       assertEquals(1, accessor.get(0));
       assertEquals(0, accessor.get(1));
 
       // test toggling the values
       m.set(0, 0);
       m.set(1, 1);
+      m.setValueCount(2);
       assertEquals(0, accessor.get(0));
       assertEquals(1, accessor.get(1));
 
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/TestSplitAndTransfer.java b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/TestSplitAndTransfer.java
index 1553a9a..057fa13 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/TestSplitAndTransfer.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/TestSplitAndTransfer.java
@@ -17,10 +17,6 @@
  */
 package org.apache.drill.exec.vector;
 
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.common.types.Types;
@@ -31,6 +27,12 @@ import org.apache.drill.exec.record.TransferPair;
 import org.apache.drill.exec.vector.NullableVarCharVector.Accessor;
 import org.junit.Test;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertArrayEquals;
+
+
 public class TestSplitAndTransfer {
   @Test
   public void test() throws Exception {
@@ -77,4 +79,102 @@ public class TestSplitAndTransfer {
     varCharVector.close();
     allocator.close();
   }
+
+  /**
+   *  BitVector tests
+   */
+
+  enum TestBitPattern {
+    ZERO,
+    ONE,
+    ALTERNATING,
+    RANDOM
+  }
+
+  @Test
+  public void testBitVectorUnalignedStart() throws Exception {
+
+    testBitVectorImpl(24, new int[][] {{5, 17}}, TestBitPattern.ONE);
+    testBitVectorImpl(24, new int[][] {{5, 17}}, TestBitPattern.ZERO);
+    testBitVectorImpl(24, new int[][] {{5, 17}}, TestBitPattern.ALTERNATING);
+    testBitVectorImpl(24, new int[][] {{5, 17}}, TestBitPattern.RANDOM);
+
+    testBitVectorImpl(3443, new int[][] {{0, 2047}, {2047, 1396}}, TestBitPattern.ZERO);
+    testBitVectorImpl(3443, new int[][] {{0, 2047}, {2047, 1396}}, TestBitPattern.ONE);
+    testBitVectorImpl(3443, new int[][] {{0, 2047}, {2047, 1396}}, TestBitPattern.ALTERNATING);
+    testBitVectorImpl(3443, new int[][] {{0, 2047}, {2047, 1396}}, TestBitPattern.RANDOM);
+
+    testBitVectorImpl(3447, new int[][] {{0, 2047}, {2047, 1400}}, TestBitPattern.ZERO);
+    testBitVectorImpl(3447, new int[][] {{0, 2047}, {2047, 1400}}, TestBitPattern.ONE);
+    testBitVectorImpl(3447, new int[][] {{0, 2047}, {2047, 1400}}, TestBitPattern.ALTERNATING);
+    testBitVectorImpl(3447, new int[][] {{0, 2047}, {2047, 1400}}, TestBitPattern.RANDOM);
+  }
+
+  @Test
+  public void testBitVectorAlignedStart() throws Exception {
+
+    testBitVectorImpl(24, new int[][] {{0, 17}}, TestBitPattern.ONE);
+    testBitVectorImpl(24, new int[][] {{0, 17}}, TestBitPattern.ZERO);
+    testBitVectorImpl(24, new int[][] {{0, 17}}, TestBitPattern.ALTERNATING);
+    testBitVectorImpl(24, new int[][] {{0, 17}}, TestBitPattern.RANDOM);
+
+    testBitVectorImpl(3444, new int[][] {{0, 2048}, {2048, 1396}}, TestBitPattern.ZERO);
+    testBitVectorImpl(3444, new int[][] {{0, 2048}, {2048, 1396}}, TestBitPattern.ONE);
+    testBitVectorImpl(3444, new int[][] {{0, 2048}, {2048, 1396}}, TestBitPattern.ALTERNATING);
+    testBitVectorImpl(3444, new int[][] {{0, 2048}, {2048, 1396}}, TestBitPattern.RANDOM);
+
+    testBitVectorImpl(3448, new int[][] {{0, 2048}, {2048, 1400}}, TestBitPattern.ZERO);
+    testBitVectorImpl(3448, new int[][] {{0, 2048}, {2048, 1400}}, TestBitPattern.ONE);
+    testBitVectorImpl(3448, new int[][] {{0, 2048}, {2048, 1400}}, TestBitPattern.ALTERNATING);
+    testBitVectorImpl(3448, new int[][] {{0, 2048}, {2048, 1400}}, TestBitPattern.RANDOM);
+  }
+
+  int getBit(TestBitPattern pattern, int index) {
+    if (pattern == TestBitPattern.RANDOM) {
+      return (int) (Math.random() * 2);
+    }
+    return (pattern == TestBitPattern.ALTERNATING) ? (index % 2) : ((pattern == TestBitPattern.ONE) ? 1 : 0);
+  }
+
+  public void testBitVectorImpl(int valueCount, final int[][] startLengths, TestBitPattern pattern) throws Exception {
+    final DrillConfig drillConfig = DrillConfig.create();
+    final BufferAllocator allocator = RootAllocatorFactory.newRoot(drillConfig);
+    final MaterializedField field = MaterializedField.create("field", Types.optional(MinorType.BIT));
+    final BitVector bitVector = new BitVector(field, allocator);
+    bitVector.allocateNew(valueCount  + 8); // extra byte at the end that gets filled with junk
+    final int[] compareArray = new int[valueCount];
+
+    int testBitValue = 0 ;
+    final BitVector.Mutator mutator = bitVector.getMutator();
+    for (int i = 0; i < valueCount; i ++) {
+      testBitValue = getBit(pattern, i);
+      mutator.set(i, testBitValue);
+      compareArray[i] = testBitValue;
+    }
+
+    // write some junk value at the end to catch
+    // off-by-one out-of-bound reads
+    for (int j = valueCount; j < valueCount + 8; j++) {
+      mutator.set(j, ~testBitValue); // fill with compliment of testBit
+    }
+    mutator.setValueCount(valueCount);
+
+    final TransferPair tp = bitVector.getTransferPair(allocator);
+    final BitVector newBitVector = (BitVector) tp.getTo();
+    final BitVector.Accessor accessor = newBitVector.getAccessor();
+
+    for (final int[] startLength : startLengths) {
+      final int start = startLength[0];
+      final int length = startLength[1];
+      tp.splitAndTransfer(start, length);
+      assertEquals(newBitVector.getAccessor().getValueCount(), length);
+      for (int i = 0; i < length; i++) {
+        final int expectedValue = compareArray[start + i];
+        assertEquals(expectedValue, accessor.get(i));
+      }
+      newBitVector.clear();
+    }
+    bitVector.close();
+    allocator.close();
+  }
 }
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/BitVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/BitVector.java
index 3725364..0dd34f5 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/BitVector.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/BitVector.java
@@ -285,20 +285,20 @@ public final class BitVector extends BaseDataValueVector implements FixedWidthVe
 
   public void splitAndTransferTo(int startIndex, int length, BitVector target) {
     assert startIndex + length <= valueCount;
-    int firstByte = getByteIndex(startIndex);
-    int byteSize = getSizeFromCount(length);
-    int offset = startIndex % 8;
-    if (offset == 0) {
+    int firstByteIndex = getByteIndex(startIndex);//byte offset of the first src byte
+    int numBytesHoldingSourceBits = getSizeFromCount(length); //src bytes to read (including start/end bytes that might not be fully copied)
+    int firstBitOffset = startIndex % 8; //Offset of first src bit within the first src byte
+    if (firstBitOffset == 0) {
       target.clear();
       // slice
       if (target.data != null) {
         target.data.release();
       }
-      target.data = data.slice(firstByte, byteSize);
+      target.data = data.slice(firstByteIndex, numBytesHoldingSourceBits);
       target.data.retain(1);
     } else {
       // Copy data
-      // When the first bit starts from the middle of a byte (offset != 0), copy data from src BitVector.
+      // When the first bit starts from the middle of a byte (firstBitOffset != 0), copy data from src BitVector.
       // Each byte in the target is composed by a part in i-th byte, another part in (i+1)-th byte.
       // The last byte copied to target is a bit tricky :
       //   1) if length requires partly byte (length % 8 !=0), copy the remaining bits only.
@@ -306,14 +306,37 @@ public final class BitVector extends BaseDataValueVector implements FixedWidthVe
       target.clear();
       target.allocateNew(length);
       // TODO maybe do this one word at a time, rather than byte?
-      for(int i = 0; i < byteSize - 1; i++) {
-        target.data.setByte(i, (((this.data.getByte(firstByte + i) & 0xFF) >>> offset) + (this.data.getByte(firstByte + i + 1) <<  (8 - offset))));
+
+      byte byteI, byteIPlus1 = 0;
+      for(int i = 0; i < numBytesHoldingSourceBits - 1; i++) {
+        byteI = this.data.getByte(firstByteIndex + i);
+        byteIPlus1 = this.data.getByte(firstByteIndex + i + 1);
+        // Extract higher-X bits from first byte i and lower-Y bits from byte (i + 1), where X + Y = 8 bits
+        // Lower-Y  bits are the MS bits of the byte to be written (target byte) and Higher-X are the LS bits.
+        // The target bytes are assembled in accordance to little-endian ordering (byte[0] = LS bit, byte[7] = MS bit)
+        target.data.setByte(i, (((byteI & 0xFF) >>> firstBitOffset) + (byteIPlus1 <<  (8 - firstBitOffset))));
       }
+
+      //Copying the last n bits
+
+      //Copy length is not a byte-multiple
       if (length % 8 != 0) {
-        target.data.setByte(byteSize - 1, ((this.data.getByte(firstByte + byteSize - 1) & 0xFF) >>> offset));
+        // start is not byte aligned so we have to copy some bits from the last full byte read in the
+        // previous loop
+        byte lastButOneByte = byteIPlus1;
+        byte bitsFromLastButOneByte = (byte)((lastButOneByte & 0xFF) >>> firstBitOffset);
+
+        // If we have to read more bits than what we have already read, read it into lastByte otherwise set lastByte to 0.
+        // (length % 8) is num of remaining bits to be read.
+        // (8 - firstBitOffset) is the number of bits already read into lastButOneByte but not used in the previous write.
+        // We do not have to read more bits if (8 - firstBitOffset >= length % 8)
+        final int lastByte = (8 - firstBitOffset >= length % 8) ?
+                0 : this.data.getByte(firstByteIndex + numBytesHoldingSourceBits);
+        target.data.setByte(numBytesHoldingSourceBits - 1, bitsFromLastButOneByte + (lastByte << (8 - firstBitOffset)));
       } else {
-        target.data.setByte(byteSize - 1,
-            (((this.data.getByte(firstByte + byteSize - 1) & 0xFF) >>> offset) + (this.data.getByte(firstByte + byteSize) <<  (8 - offset))));
+        target.data.setByte(numBytesHoldingSourceBits - 1,
+            (((this.data.getByte(firstByteIndex + numBytesHoldingSourceBits - 1) & 0xFF) >>> firstBitOffset) +
+                     (this.data.getByte(firstByteIndex + numBytesHoldingSourceBits) <<  (8 - firstBitOffset))));
       }
     }
     target.getMutator().setValueCount(length);


[drill] 02/09: DRILL-6470: Remove defunct repository

Posted by vi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit fc825c5eff56a8433a967dd2dc41422442b68aaa
Author: Kunal Khatua <ku...@apache.org>
AuthorDate: Mon Jun 18 10:36:43 2018 -0700

    DRILL-6470: Remove defunct repository
    
    When build the Drill source code, Parquet jars from the repository hosted at http://repo.dremio.com/release/  is inaccessible.
    These artifacts (parquet libraries) are now available within http://maven.corp.maprtech.com/nexus/content/groups/public/org/apache/parquet/
    
    closes #1326
---
 pom.xml | 12 ------------
 1 file changed, 12 deletions(-)

diff --git a/pom.xml b/pom.xml
index f69288e..30b9129 100644
--- a/pom.xml
+++ b/pom.xml
@@ -166,18 +166,6 @@
     </repository>
 
     <repository>
-      <id>dremio-releases</id>
-      <name>Dremio Drill Third Party Artifacts</name>
-      <url>http://repo.dremio.com/release/</url>
-      <releases>
-        <enabled>true</enabled>
-      </releases>
-      <snapshots>
-        <enabled>false</enabled>
-      </snapshots>
-    </repository>
-
-    <repository>
       <id>mapr-drill-thirdparty</id>
       <name>MapR Drill Third Party Artifacts</name>
       <url>http://repository.mapr.com/nexus/content/repositories/drill/</url>


[drill] 06/09: DRILL-6523: Fix NPE for describe of partial schema

Posted by vi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit aa127b70b1e46f7f4aa19881f25eda583627830a
Author: Arina Ielchiieva <ar...@gmail.com>
AuthorDate: Thu Jun 21 17:04:13 2018 +0300

    DRILL-6523: Fix NPE for describe of partial schema
    
    closes #1332
---
 .../sql/handlers/DescribeSchemaHandler.java        | 53 ++++++++++++----------
 .../org/apache/drill/exec/sql/TestInfoSchema.java  |  8 +++-
 2 files changed, 37 insertions(+), 24 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DescribeSchemaHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DescribeSchemaHandler.java
index f97696e..bb51ef0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DescribeSchemaHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DescribeSchemaHandler.java
@@ -32,10 +32,12 @@ import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.planner.sql.DirectPlan;
 import org.apache.drill.exec.planner.sql.SchemaUtilites;
+import org.apache.drill.exec.store.AbstractSchema;
 import org.apache.drill.exec.store.StoragePlugin;
 import org.apache.drill.exec.store.dfs.FileSystemPlugin;
 import org.apache.drill.exec.store.dfs.FileSystemSchemaFactory;
 import org.apache.drill.exec.store.dfs.WorkspaceConfig;
+import org.apache.drill.exec.work.foreman.ForemanSetupException;
 
 import java.util.List;
 import java.util.Map;
@@ -68,33 +70,38 @@ public class DescribeSchemaHandler extends DefaultSqlHandler {
 
 
   @Override
-  public PhysicalPlan getPlan(SqlNode sqlNode) {
-    SqlIdentifier schema = ((SqlDescribeSchema) sqlNode).getSchema();
-    SchemaPlus drillSchema = SchemaUtilites.findSchema(config.getConverter().getDefaultSchema(), schema.names);
-
-    if (drillSchema != null) {
-      StoragePlugin storagePlugin;
-      try {
-        storagePlugin = context.getStorage().getPlugin(schema.names.get(0));
-      } catch (ExecutionSetupException e) {
-        throw new DrillRuntimeException("Failure while retrieving storage plugin", e);
+  public PhysicalPlan getPlan(SqlNode sqlNode) throws ForemanSetupException {
+    SqlIdentifier schema = unwrap(sqlNode, SqlDescribeSchema.class).getSchema();
+    SchemaPlus schemaPlus = SchemaUtilites.findSchema(config.getConverter().getDefaultSchema(), schema.names);
+
+    if (schemaPlus == null) {
+      throw UserException.validationError()
+        .message("Invalid schema name [%s]", Joiner.on(".").join(schema.names))
+        .build(logger);
+    }
+
+    StoragePlugin storagePlugin;
+    try {
+      AbstractSchema drillSchema = SchemaUtilites.unwrapAsDrillSchemaInstance(schemaPlus);
+      storagePlugin = context.getStorage().getPlugin(drillSchema.getSchemaPath().get(0));
+      if (storagePlugin == null) {
+        throw new DrillRuntimeException(String.format("Unable to find storage plugin with the following name [%s].",
+          drillSchema.getSchemaPath().get(0)));
       }
-      String properties;
-      try {
-        final Map configMap = mapper.convertValue(storagePlugin.getConfig(), Map.class);
-        if (storagePlugin instanceof FileSystemPlugin) {
-          transformWorkspaces(schema.names, configMap);
-        }
-        properties = mapper.writeValueAsString(configMap);
-      } catch (JsonProcessingException e) {
-        throw new DrillRuntimeException("Error while trying to convert storage config to json string", e);
+    } catch (ExecutionSetupException e) {
+      throw new DrillRuntimeException("Failure while retrieving storage plugin", e);
+    }
+
+    try {
+      Map configMap = mapper.convertValue(storagePlugin.getConfig(), Map.class);
+      if (storagePlugin instanceof FileSystemPlugin) {
+        transformWorkspaces(schema.names, configMap);
       }
+      String properties = mapper.writeValueAsString(configMap);
       return DirectPlan.createDirectPlan(context, new DescribeSchemaResult(Joiner.on(".").join(schema.names), properties));
+    } catch (JsonProcessingException e) {
+      throw new DrillRuntimeException("Error while trying to convert storage config to json string", e);
     }
-
-    throw UserException.validationError()
-          .message(String.format("Invalid schema name [%s]", Joiner.on(".").join(schema.names)))
-          .build(logger);
   }
 
   /**
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestInfoSchema.java b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestInfoSchema.java
index a702574..e0ed2fb 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestInfoSchema.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestInfoSchema.java
@@ -383,9 +383,15 @@ public class TestInfoSchema extends BaseTestQuery {
   }
 
   @Test
+  public void describePartialSchema() throws Exception {
+    test("use dfs");
+    test("describe schema tmp");
+  }
+
+  @Test
   public void describeSchemaOutput() throws Exception {
     final List<QueryDataBatch> result = testSqlWithResults("describe schema dfs.tmp");
-    assertTrue(result.size() == 1);
+    assertEquals(1, result.size());
     final QueryDataBatch batch = result.get(0);
     final RecordBatchLoader loader = new RecordBatchLoader(getDrillbitContext().getAllocator());
     loader.load(batch.getHeader().getDef(), batch.getData());


[drill] 03/09: DRILL-6491: Prevent merge join for full outer join at planning stage

Posted by vi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 9299fcc29dd9ac320b62b986f74423e8e6d1b750
Author: Bohdan Kazydub <bo...@gmail.com>
AuthorDate: Tue Jun 12 18:58:24 2018 +0300

    DRILL-6491: Prevent merge join for full outer join at planning stage
    
    closes #1320
---
 .../drill/exec/planner/physical/MergeJoinPrel.java |   3 +-
 .../physical/impl/join/TestMergeJoinAdvanced.java  | 160 +++++++++++++--------
 2 files changed, 99 insertions(+), 64 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
index 52e8921..2928be2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
@@ -65,7 +65,8 @@ public class MergeJoinPrel  extends JoinPrel {
     if (PrelUtil.getSettings(getCluster()).useDefaultCosting()) {
       return super.computeSelfCost(planner, mq).multiplyBy(.1);
     }
-    if (joincategory == JoinCategory.CARTESIAN || joincategory == JoinCategory.INEQUALITY) {
+    if (joincategory == JoinCategory.CARTESIAN || joincategory == JoinCategory.INEQUALITY
+        || getJoinType() == JoinRelType.FULL) {
       return planner.getCostFactory().makeInfiniteCost();
     }
     double leftRowCount = mq.getRowCount(this.getLeft());
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoinAdvanced.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoinAdvanced.java
index 147323d..d8bdea9 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoinAdvanced.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoinAdvanced.java
@@ -17,16 +17,20 @@
  */
 package org.apache.drill.exec.physical.impl.join;
 
+import org.apache.drill.common.exceptions.UserRemoteException;
 import org.apache.drill.test.TestTools;
 import org.apache.drill.categories.OperatorTest;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
-import org.junit.AfterClass;
+import org.hamcrest.CoreMatchers;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.Ignore;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.ExpectedException;
 import org.junit.rules.TestRule;
 
 import java.io.BufferedWriter;
@@ -48,67 +52,69 @@ public class TestMergeJoinAdvanced extends JoinTestBase {
 
   @Rule
   public final TestRule TIMEOUT = TestTools.getTimeoutRule(120000); // Longer timeout than usual.
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
 
-  // Have to disable hash join to test merge join in this class
   @BeforeClass
-  public static void enableMergeJoin() throws Exception {
-    test(DISABLE_HJ);
-
+  public static void init() {
     leftFile = new File(dirTestWatcher.getRootDir(), LEFT);
     rightFile = new File(dirTestWatcher.getRootDir(), RIGHT);
 
     dirTestWatcher.copyResourceToRoot(Paths.get("join"));
   }
 
-  @AfterClass
-  public static void disableMergeJoin() throws Exception {
-    test(ENABLE_HJ);
+  // Have to disable hash join to test merge join in this class
+  @Before
+  public void disableHashJoin() throws Exception {
+    test(DISABLE_HJ);
+  }
+
+  @After
+  public void enableHashJoin() throws Exception {
+    test(RESET_HJ);
   }
 
   @Test
   public void testJoinWithDifferentTypesInCondition() throws Exception {
-    String query = "select t1.full_name from cp.`employee.json` t1, cp.`department.json` t2 " +
-        "where cast(t1.department_id as double) = t2.department_id and t1.employee_id = 1";
-
-    testBuilder()
-        .sqlQuery(query)
-        .optionSettingQueriesForTestQuery(ENABLE_HJ)
-        .unOrdered()
-        .baselineColumns("full_name")
-        .baselineValues("Sheri Nowmer")
-        .go();
-
-
-    query = "select t1.bigint_col from cp.`jsoninput/implicit_cast_join_1.json` t1, cp.`jsoninput/implicit_cast_join_1.json` t2 " +
-        " where t1.bigint_col = cast(t2.bigint_col as int) and" + // join condition with bigint and int
-        " t1.double_col  = cast(t2.double_col as float) and" + // join condition with double and float
-        " t1.bigint_col = cast(t2.bigint_col as double)"; // join condition with bigint and double
-
-    testBuilder()
-        .sqlQuery(query)
-        .optionSettingQueriesForTestQuery(ENABLE_HJ)
-        .unOrdered()
-        .baselineColumns("bigint_col")
-        .baselineValues(1l)
-        .go();
-
-    query = "select count(*) col1 from " +
+    String query = "select count(*) col1 from " +
         "(select t1.date_opt from cp.`parquet/date_dictionary.parquet` t1, cp.`parquet/timestamp_table.parquet` t2 " +
         "where t1.date_opt = t2.timestamp_col)"; // join condition contains date and timestamp
-
-    testBuilder()
-        .sqlQuery(query)
+    testBuilder().sqlQuery(query)
         .unOrdered()
         .baselineColumns("col1")
-        .baselineValues(4l)
+        .baselineValues(4L)
         .go();
+
+    try {
+      test(ENABLE_HJ);
+
+      query = "select t1.full_name from cp.`employee.json` t1, cp.`department.json` t2 " +
+          "where cast(t1.department_id as double) = t2.department_id and t1.employee_id = 1";
+      testBuilder().sqlQuery(query)
+          .unOrdered()
+          .baselineColumns("full_name")
+          .baselineValues("Sheri Nowmer")
+          .go();
+
+
+      query = "select t1.bigint_col from cp.`jsoninput/implicit_cast_join_1.json` t1, cp.`jsoninput/implicit_cast_join_1.json` t2 " +
+          " where t1.bigint_col = cast(t2.bigint_col as int) and" + // join condition with bigint and int
+          " t1.double_col  = cast(t2.double_col as float) and" + // join condition with double and float
+          " t1.bigint_col = cast(t2.bigint_col as double)"; // join condition with bigint and double
+      testBuilder().sqlQuery(query)
+          .unOrdered()
+          .baselineColumns("bigint_col")
+          .baselineValues(1L)
+          .go();
+    } finally {
+      test(RESET_HJ);
+    }
   }
 
   @Test
   @Ignore // TODO file JIRA to fix this
   public void testFix2967() throws Exception {
     setSessionOption(PlannerSettings.BROADCAST.getOptionName(), "false");
-    setSessionOption(PlannerSettings.HASHJOIN.getOptionName(), "false");
     setSessionOption(ExecConstants.SLICE_TARGET, "1");
     setSessionOption(ExecConstants.MAX_WIDTH_PER_NODE_KEY, "23");
 
@@ -143,11 +149,10 @@ public class TestMergeJoinAdvanced extends JoinTestBase {
     final BufferedWriter leftWriter = new BufferedWriter(new FileWriter(leftFile));
     final BufferedWriter rightWriter = new BufferedWriter(new FileWriter(rightFile));
     generateData(leftWriter, rightWriter, left, right);
-    final String query1 = String.format("select count(*) c1 from dfs.`%s` L %s join dfs.`%s` R on L.k=R.k1",
+    final String query = String.format("select count(*) c1 from dfs.`%s` L %s join dfs.`%s` R on L.k=R.k1",
       LEFT, joinType, RIGHT);
     testBuilder()
-      .sqlQuery(query1)
-      .optionSettingQueriesForTestQuery(DISABLE_HJ)
+      .sqlQuery(query)
       .unOrdered()
       .baselineColumns("c1")
       .baselineValues(expected)
@@ -156,32 +161,32 @@ public class TestMergeJoinAdvanced extends JoinTestBase {
 
   @Test
   public void testMergeInnerJoinLargeRight() throws Exception {
-    testMultipleBatchJoin(1000l, 5000l, "inner", 5000l * 1000l);
+    testMultipleBatchJoin(1000L, 5000L, "inner", 5000L * 1000L);
   }
 
   @Test
   public void testMergeLeftJoinLargeRight() throws Exception {
-    testMultipleBatchJoin(1000l, 5000l, "left", 5000l * 1000l +2l);
+    testMultipleBatchJoin(1000L, 5000L, "left", 5000L * 1000L +2L);
   }
 
   @Test
   public void testMergeRightJoinLargeRight() throws Exception {
-    testMultipleBatchJoin(1000l, 5000l, "right", 5000l*1000l +3l);
+    testMultipleBatchJoin(1000L, 5000L, "right", 5000L * 1000L +3L);
   }
 
   @Test
   public void testMergeInnerJoinLargeLeft() throws Exception {
-    testMultipleBatchJoin(5000l, 1000l, "inner", 5000l*1000l);
+    testMultipleBatchJoin(5000L, 1000L, "inner", 5000L * 1000L);
   }
 
   @Test
   public void testMergeLeftJoinLargeLeft() throws Exception {
-    testMultipleBatchJoin(5000l, 1000l, "left", 5000l*1000l + 2l);
+    testMultipleBatchJoin(5000L, 1000L, "left", 5000L * 1000L + 2L);
   }
 
   @Test
   public void testMergeRightJoinLargeLeft() throws Exception {
-    testMultipleBatchJoin(5000l, 1000l, "right", 5000l*1000l + 3l);
+    testMultipleBatchJoin(5000L, 1000L, "right", 5000L * 1000L + 3L);
   }
 
   // Following tests can take some time.
@@ -189,37 +194,38 @@ public class TestMergeJoinAdvanced extends JoinTestBase {
   @Ignore
   public void testMergeInnerJoinRandomized() throws Exception {
     final Random r = new Random();
-    final long right = r.nextInt(10001) + 1l;
-    final long left = r.nextInt(10001) + 1l;
-    testMultipleBatchJoin(left, right, "inner", left*right);
+    final long right = r.nextInt(10001) + 1L;
+    final long left = r.nextInt(10001) + 1L;
+    testMultipleBatchJoin(left, right, "inner", left * right);
   }
 
   @Test
   @Ignore
   public void testMergeLeftJoinRandomized() throws Exception {
     final Random r = new Random();
-    final long right = r.nextInt(10001) + 1l;
-    final long left = r.nextInt(10001) + 1l;
-    testMultipleBatchJoin(left, right, "left", left*right + 2l);
+    final long right = r.nextInt(10001) + 1L;
+    final long left = r.nextInt(10001) + 1L;
+    testMultipleBatchJoin(left, right, "left", left * right + 2L);
   }
 
   @Test
   @Ignore
   public void testMergeRightJoinRandomized() throws Exception {
     final Random r = new Random();
-    final long right = r.nextInt(10001) + 1l;
-    final long left = r.nextInt(10001) + 1l;
-    testMultipleBatchJoin(left, right, "right", left * right + 3l);
+    final long right = r.nextInt(10001) + 1L;
+    final long left = r.nextInt(10001) + 1L;
+    testMultipleBatchJoin(left, right, "right", left * right + 3L);
   }
 
   @Test
   public void testDrill4165() throws Exception {
-    final String query1 = "select count(*) cnt from cp.`tpch/lineitem.parquet` l1, cp.`tpch/lineitem.parquet` l2 where l1.l_partkey = l2.l_partkey and l1.l_suppkey < 30 and l2.l_suppkey < 30";
+    final String query = "select count(*) cnt from cp.`tpch/lineitem.parquet` l1, cp.`tpch/lineitem.parquet` l2 " +
+        "where l1.l_partkey = l2.l_partkey and l1.l_suppkey < 30 and l2.l_suppkey < 30";
     testBuilder()
-      .sqlQuery(query1)
+      .sqlQuery(query)
       .unOrdered()
       .baselineColumns("cnt")
-      .baselineValues(202452l)
+      .baselineValues(202452L)
       .go();
   }
 
@@ -244,11 +250,10 @@ public class TestMergeJoinAdvanced extends JoinTestBase {
     leftWriter.close();
     rightWriter.close();
 
-    final String query1 = String.format("select count(*) c1 from dfs.`%s` L %s join dfs.`%s` R on L.k=R.k1",
+    final String query = String.format("select count(*) c1 from dfs.`%s` L %s join dfs.`%s` R on L.k=R.k1",
       LEFT, "inner", RIGHT);
     testBuilder()
-      .sqlQuery(query1)
-      .optionSettingQueriesForTestQuery(DISABLE_HJ)
+      .sqlQuery(query)
       .unOrdered()
       .baselineColumns("c1")
       .baselineValues(6000*800L)
@@ -269,4 +274,33 @@ public class TestMergeJoinAdvanced extends JoinTestBase {
   public void testMergeRightJoinWithEmptyTable() throws Exception {
     testJoinWithEmptyFile(dirTestWatcher.getRootDir(), "right outer", new String[] {MJ_PATTERN, RIGHT_JOIN_TYPE}, 0L);
   }
+
+  @Test // DRILL-6491
+  public void testMergeIsNotSelectedForFullJoin() throws Exception {
+    try {
+      test(ENABLE_HJ);
+
+      String query = "select * " +
+          " from (select employee_id from cp.`employee.json` order by employee_id) e1 " +
+          " full outer join (select employee_id from cp.`employee.json` order by employee_id) e2 " +
+          " on e1.employee_id = e2.employee_id " +
+          " limit 10";
+      testPlanMatchingPatterns(query, null, new String[]{MJ_PATTERN});
+    } finally {
+      test(RESET_HJ);
+    }
+  }
+
+  @Test // DRILL-6491
+  public void testFullJoinIsNotSupported() throws Exception {
+    thrown.expect(UserRemoteException.class);
+    thrown.expectMessage(CoreMatchers.containsString("SYSTEM ERROR: CannotPlanException"));
+
+    String query = "select * " +
+        " from (select employee_id from cp.`employee.json` order by employee_id) e1 " +
+        " full outer join (select employee_id from cp.`employee.json` order by employee_id) e2 " +
+        " on e1.employee_id = e2.employee_id " +
+        " limit 10";
+    test(query);
+  }
 }


[drill] 04/09: DRILL-6502: Rename CorrelatePrel to LateralJoinPrel.

Posted by vi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 9c7e55798259ff4ac1aca102539731348e2c9bd3
Author: Hanumath Rao Maduri <ha...@gmail.com>
AuthorDate: Fri Jun 15 12:05:09 2018 -0700

    DRILL-6502: Rename CorrelatePrel to LateralJoinPrel.
    
    closes #1325
---
 .../apache/drill/exec/planner/PlannerPhase.java    |  4 ++--
 ...teRelBase.java => DrillLateralJoinRelBase.java} |  6 +++---
 .../exec/planner/logical/DrillCorrelateRule.java   |  6 +++---
 ...lCorrelateRel.java => DrillLateralJoinRel.java} | 10 ++++-----
 .../{CorrelatePrel.java => LateralJoinPrel.java}   | 14 ++++++------
 .../{CorrelatePrule.java => LateralJoinPrule.java} | 25 +++++++++++-----------
 .../drill/exec/planner/physical/UnnestPrel.java    |  2 +-
 .../physical/explain/NumberingRelWriter.java       | 20 ++++++++---------
 .../planner/physical/visitor/BasePrelVisitor.java  |  4 ++--
 .../visitor/ExcessiveExchangeIdentifier.java       |  6 +++---
 .../physical/visitor/JoinPrelRenameVisitor.java    |  8 +++----
 .../exec/planner/physical/visitor/PrelVisitor.java | 22 +++++++++----------
 .../physical/visitor/PrelVisualizerVisitor.java    |  4 ++--
 .../impl/lateraljoin/TestLateralPlans.java         |  6 +++---
 14 files changed, 68 insertions(+), 69 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java
index b78d76c..c8bb2a4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java
@@ -55,7 +55,7 @@ import org.apache.drill.exec.planner.logical.DrillWindowRule;
 import org.apache.drill.exec.planner.logical.partition.ParquetPruneScanRule;
 import org.apache.drill.exec.planner.logical.partition.PruneScanRule;
 import org.apache.drill.exec.planner.physical.ConvertCountToDirectScan;
-import org.apache.drill.exec.planner.physical.CorrelatePrule;
+import org.apache.drill.exec.planner.physical.LateralJoinPrule;
 import org.apache.drill.exec.planner.physical.DirectScanPrule;
 import org.apache.drill.exec.planner.physical.FilterPrule;
 import org.apache.drill.exec.planner.physical.HashAggPrule;
@@ -461,7 +461,7 @@ public enum PlannerPhase {
     ruleList.add(DirectScanPrule.INSTANCE);
 
     ruleList.add(UnnestPrule.INSTANCE);
-    ruleList.add(CorrelatePrule.INSTANCE);
+    ruleList.add(LateralJoinPrule.INSTANCE);
 
     ruleList.add(DrillPushLimitToScanRule.LIMIT_ON_PROJECT);
     ruleList.add(DrillPushLimitToScanRule.LIMIT_ON_SCAN);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillCorrelateRelBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillLateralJoinRelBase.java
similarity index 87%
rename from exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillCorrelateRelBase.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillLateralJoinRelBase.java
index ea994ba..a7bbbca 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillCorrelateRelBase.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillLateralJoinRelBase.java
@@ -32,9 +32,9 @@ import org.apache.drill.exec.planner.cost.DrillCostBase;
 import org.apache.drill.exec.planner.physical.PrelUtil;
 
 
-public abstract class DrillCorrelateRelBase extends Correlate implements DrillRelNode {
-  public DrillCorrelateRelBase(RelOptCluster cluster, RelTraitSet traits, RelNode left, RelNode right,
-                               CorrelationId correlationId, ImmutableBitSet requiredColumns, SemiJoinType semiJoinType) {
+public abstract class DrillLateralJoinRelBase extends Correlate implements DrillRelNode {
+  public DrillLateralJoinRelBase(RelOptCluster cluster, RelTraitSet traits, RelNode left, RelNode right,
+                                 CorrelationId correlationId, ImmutableBitSet requiredColumns, SemiJoinType semiJoinType) {
     super(cluster, traits, left, right, correlationId, requiredColumns, semiJoinType);
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillCorrelateRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillCorrelateRule.java
index 8ac4fb1..52e603f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillCorrelateRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillCorrelateRule.java
@@ -33,7 +33,7 @@ public class DrillCorrelateRule extends RelOptRule {
   private DrillCorrelateRule() {
     super(RelOptHelper.any(LogicalCorrelate.class, Convention.NONE),
         DrillRelFactories.LOGICAL_BUILDER,
-        "DrillCorrelateRule");
+        "DrillLateralJoinRule");
   }
 
   @Override
@@ -45,9 +45,9 @@ public class DrillCorrelateRule extends RelOptRule {
     final RelNode convertedRight = convert(right, right.getTraitSet().plus(DrillRel.DRILL_LOGICAL).simplify());
 
     final RelTraitSet traits = correlate.getTraitSet().plus(DrillRel.DRILL_LOGICAL);
-    DrillCorrelateRel correlateRel = new DrillCorrelateRel(correlate.getCluster(),
+    DrillLateralJoinRel lateralJoinRel = new DrillLateralJoinRel(correlate.getCluster(),
         traits, convertedLeft, convertedRight, correlate.getCorrelationId(),
         correlate.getRequiredColumns(), correlate.getJoinType());
-    call.transformTo(correlateRel);
+    call.transformTo(lateralJoinRel);
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillCorrelateRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillLateralJoinRel.java
similarity index 80%
rename from exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillCorrelateRel.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillLateralJoinRel.java
index 7c49232..035dae9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillCorrelateRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillLateralJoinRel.java
@@ -26,15 +26,15 @@ import org.apache.calcite.sql.SemiJoinType;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.drill.common.logical.data.LateralJoin;
 import org.apache.drill.common.logical.data.LogicalOperator;
-import org.apache.drill.exec.planner.common.DrillCorrelateRelBase;
+import org.apache.drill.exec.planner.common.DrillLateralJoinRelBase;
 
 import java.util.List;
 
 
-public class DrillCorrelateRel extends DrillCorrelateRelBase implements DrillRel {
+public class DrillLateralJoinRel extends DrillLateralJoinRelBase implements DrillRel {
 
-  protected DrillCorrelateRel(RelOptCluster cluster, RelTraitSet traits, RelNode left, RelNode right,
-                              CorrelationId correlationId, ImmutableBitSet requiredColumns, SemiJoinType semiJoinType) {
+  protected DrillLateralJoinRel(RelOptCluster cluster, RelTraitSet traits, RelNode left, RelNode right,
+                                CorrelationId correlationId, ImmutableBitSet requiredColumns, SemiJoinType semiJoinType) {
     super(cluster, traits, left, right, correlationId, requiredColumns, semiJoinType);
   }
 
@@ -42,7 +42,7 @@ public class DrillCorrelateRel extends DrillCorrelateRelBase implements DrillRel
   public Correlate copy(RelTraitSet traitSet,
         RelNode left, RelNode right, CorrelationId correlationId,
         ImmutableBitSet requiredColumns, SemiJoinType joinType) {
-    return new DrillCorrelateRel(this.getCluster(), this.getTraitSet(), left, right, correlationId, requiredColumns,
+    return new DrillLateralJoinRel(this.getCluster(), this.getTraitSet(), left, right, correlationId, requiredColumns,
         this.getJoinType());
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/CorrelatePrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/LateralJoinPrel.java
similarity index 88%
rename from exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/CorrelatePrel.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/LateralJoinPrel.java
index 9938db1..565871b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/CorrelatePrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/LateralJoinPrel.java
@@ -32,7 +32,7 @@ import org.apache.calcite.sql.SemiJoinType;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.config.LateralJoinPOP;
-import org.apache.drill.exec.planner.common.DrillCorrelateRelBase;
+import org.apache.drill.exec.planner.common.DrillLateralJoinRelBase;
 import org.apache.drill.exec.planner.common.DrillJoinRelBase;
 import org.apache.drill.exec.planner.physical.visitor.PrelVisitor;
 import org.apache.drill.exec.record.BatchSchema;
@@ -41,18 +41,18 @@ import java.io.IOException;
 import java.util.Iterator;
 import java.util.List;
 
-public class CorrelatePrel extends DrillCorrelateRelBase implements Prel {
+public class LateralJoinPrel extends DrillLateralJoinRelBase implements Prel {
 
 
-  protected CorrelatePrel(RelOptCluster cluster, RelTraitSet traits, RelNode left, RelNode right,
-                              CorrelationId correlationId, ImmutableBitSet requiredColumns, SemiJoinType semiJoinType) {
+  protected LateralJoinPrel(RelOptCluster cluster, RelTraitSet traits, RelNode left, RelNode right,
+                            CorrelationId correlationId, ImmutableBitSet requiredColumns, SemiJoinType semiJoinType) {
     super(cluster, traits, left, right, correlationId, requiredColumns, semiJoinType);
   }
   @Override
   public Correlate copy(RelTraitSet traitSet,
                         RelNode left, RelNode right, CorrelationId correlationId,
                         ImmutableBitSet requiredColumns, SemiJoinType joinType) {
-    return new CorrelatePrel(this.getCluster(), this.getTraitSet(), left, right, correlationId, requiredColumns,
+    return new LateralJoinPrel(this.getCluster(), this.getTraitSet(), left, right, correlationId, requiredColumns,
         this.getJoinType());
   }
 
@@ -72,7 +72,7 @@ public class CorrelatePrel extends DrillCorrelateRelBase implements Prel {
    * Check to make sure that the fields of the inputs are the same as the output field names.
    * If not, insert a project renaming them.
    */
-  public RelNode getCorrelateInput(int offset, RelNode input) {
+  public RelNode getLateralInput(int offset, RelNode input) {
     Preconditions.checkArgument(DrillJoinRelBase.uniqueFieldNames(input.getRowType()));
     final List<String> fields = getRowType().getFieldNames();
     final List<String> inputFields = input.getRowType().getFieldNames();
@@ -106,7 +106,7 @@ public class CorrelatePrel extends DrillCorrelateRelBase implements Prel {
 
   @Override
   public <T, X, E extends Throwable> T accept(PrelVisitor<T, X, E> visitor, X value) throws E {
-    return visitor.visitCorrelate(this, value);
+    return visitor.visitLateral(this, value);
   }
 
   @Override
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/CorrelatePrule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/LateralJoinPrule.java
similarity index 69%
rename from exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/CorrelatePrule.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/LateralJoinPrule.java
index 4f1e1d8..e531dca 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/CorrelatePrule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/LateralJoinPrule.java
@@ -22,22 +22,22 @@ import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptRuleOperand;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelNode;
-import org.apache.drill.exec.planner.logical.DrillCorrelateRel;
+import org.apache.drill.exec.planner.logical.DrillLateralJoinRel;
 import org.apache.drill.exec.planner.logical.RelOptHelper;
 
-public class CorrelatePrule extends Prule {
-  public static final RelOptRule INSTANCE = new CorrelatePrule("Prel.CorrelatePrule",
-      RelOptHelper.any(DrillCorrelateRel.class));
+public class LateralJoinPrule extends Prule {
+  public static final RelOptRule INSTANCE = new LateralJoinPrule("Prel.LateralJoinPrule",
+      RelOptHelper.any(DrillLateralJoinRel.class));
 
-  private CorrelatePrule(String name, RelOptRuleOperand operand) {
+  private LateralJoinPrule(String name, RelOptRuleOperand operand) {
     super(operand, name);
   }
 
   @Override
   public void onMatch(RelOptRuleCall call) {
-    final DrillCorrelateRel correlate = call.rel(0);
-    final RelNode left = correlate.getLeft();
-    final RelNode right = correlate.getRight();
+    final DrillLateralJoinRel lateralJoinRel = call.rel(0);
+    final RelNode left = lateralJoinRel.getLeft();
+    final RelNode right = lateralJoinRel.getRight();
     RelTraitSet traitsLeft = left.getTraitSet().plus(Prel.DRILL_PHYSICAL);
     RelTraitSet traitsRight = right.getTraitSet().plus(Prel.DRILL_PHYSICAL);
 
@@ -46,11 +46,10 @@ public class CorrelatePrule extends Prule {
     final RelNode convertedLeft = convert(left, traitsLeft);
     final RelNode convertedRight = convert(right, traitsRight);
 
-    final CorrelatePrel correlatePrel = new CorrelatePrel(correlate.getCluster(),
+    final LateralJoinPrel lateralJoinPrel = new LateralJoinPrel(lateralJoinRel.getCluster(),
                                   corrTraits,
-                                  convertedLeft, convertedRight, correlate.getCorrelationId(),
-                                  correlate.getRequiredColumns(),correlate.getJoinType());
-    call.transformTo(correlatePrel);
+                                  convertedLeft, convertedRight, lateralJoinRel.getCorrelationId(),
+                                  lateralJoinRel.getRequiredColumns(),lateralJoinRel.getJoinType());
+    call.transformTo(lateralJoinPrel);
   }
-
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnnestPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnnestPrel.java
index a22beea..692b3d2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnnestPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnnestPrel.java
@@ -76,6 +76,6 @@ public class UnnestPrel extends DrillUnnestRelBase implements Prel {
   }
 
   public Class<?> getParentClass() {
-    return CorrelatePrel.class;
+    return LateralJoinPrel.class;
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/explain/NumberingRelWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/explain/NumberingRelWriter.java
index 38b97b6..43e0300 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/explain/NumberingRelWriter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/explain/NumberingRelWriter.java
@@ -32,7 +32,7 @@ import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.runtime.FlatLists;
 import org.apache.calcite.sql.SqlExplainLevel;
 import org.apache.calcite.util.Pair;
-import org.apache.drill.exec.planner.physical.CorrelatePrel;
+import org.apache.drill.exec.planner.physical.LateralJoinPrel;
 import org.apache.drill.exec.planner.physical.HashJoinPrel;
 import org.apache.drill.exec.planner.physical.Prel;
 import org.apache.drill.exec.planner.physical.UnnestPrel;
@@ -138,7 +138,7 @@ class NumberingRelWriter implements RelWriter {
 
   private String getDependentSrcOp(UnnestPrel unnest) {
     Prel parent = this.getRegisteredPrel(unnest.getParentClass());
-    if (parent != null && parent instanceof CorrelatePrel) {
+    if (parent != null && parent instanceof LateralJoinPrel) {
       OpId id = ids.get(parent);
       return String.format(" [srcOp=%02d-%02d] ", id.fragmentId, id.opId);
     }
@@ -159,8 +159,8 @@ class NumberingRelWriter implements RelWriter {
 
 
   private void explainInputs(RelNode rel) {
-    if (rel instanceof CorrelatePrel) {
-      this.explainInputs((CorrelatePrel) rel);
+    if (rel instanceof LateralJoinPrel) {
+      this.explainInputs((LateralJoinPrel) rel);
     } else {
       List<RelNode> inputs = rel.getInputs();
       if (rel instanceof HashJoinPrel && ((HashJoinPrel) rel).isSwapped()) {
@@ -173,13 +173,13 @@ class NumberingRelWriter implements RelWriter {
     }
   }
 
-  //Correlate is handled differently because explain plan
+  //Lateral is handled differently because explain plan
   //needs to show relation between Lateral and Unnest operators.
-  private void explainInputs(CorrelatePrel correlate) {
-    correlate.getInput(0).explain(this);
-    this.register(correlate);
-    correlate.getInput(1).explain(this);
-    this.unRegister(correlate);
+  private void explainInputs(LateralJoinPrel lateralJoinPrel) {
+    lateralJoinPrel.getInput(0).explain(this);
+    this.register(lateralJoinPrel);
+    lateralJoinPrel.getInput(1).explain(this);
+    this.unRegister(lateralJoinPrel);
   }
 
   public final void explain(RelNode rel, List<Pair<String, Object>> valueList) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/BasePrelVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/BasePrelVisitor.java
index 04b7c18..818ec35 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/BasePrelVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/BasePrelVisitor.java
@@ -25,7 +25,7 @@ import org.apache.drill.exec.planner.physical.ScanPrel;
 import org.apache.drill.exec.planner.physical.ScreenPrel;
 import org.apache.drill.exec.planner.physical.WriterPrel;
 import org.apache.drill.exec.planner.physical.UnnestPrel;
-import org.apache.drill.exec.planner.physical.CorrelatePrel;
+import org.apache.drill.exec.planner.physical.LateralJoinPrel;
 
 public class BasePrelVisitor<RETURN, EXTRA, EXCEP extends Throwable> implements PrelVisitor<RETURN, EXTRA, EXCEP> {
 
@@ -71,7 +71,7 @@ public class BasePrelVisitor<RETURN, EXTRA, EXCEP extends Throwable> implements
   }
 
   @Override
-  public RETURN visitCorrelate(CorrelatePrel prel, EXTRA value) throws EXCEP {
+  public RETURN visitLateral(LateralJoinPrel prel, EXTRA value) throws EXCEP {
     return visitPrel(prel, value);
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/ExcessiveExchangeIdentifier.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/ExcessiveExchangeIdentifier.java
index b4ed5e0..92505c2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/ExcessiveExchangeIdentifier.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/ExcessiveExchangeIdentifier.java
@@ -20,7 +20,7 @@ package org.apache.drill.exec.planner.physical.visitor;
 import java.util.Collections;
 import java.util.List;
 import org.apache.drill.exec.planner.fragment.DistributionAffinity;
-import org.apache.drill.exec.planner.physical.CorrelatePrel;
+import org.apache.drill.exec.planner.physical.LateralJoinPrel;
 import org.apache.drill.exec.planner.physical.ExchangePrel;
 import org.apache.drill.exec.planner.physical.Prel;
 import org.apache.drill.exec.planner.physical.ScanPrel;
@@ -32,7 +32,7 @@ import org.apache.drill.exec.planner.physical.UnnestPrel;
 
 public class ExcessiveExchangeIdentifier extends BasePrelVisitor<Prel, ExcessiveExchangeIdentifier.MajorFragmentStat, RuntimeException> {
   private final long targetSliceSize;
-  private CorrelatePrel topMostLateralJoin = null;
+  private LateralJoinPrel topMostLateralJoin = null;
 
   public ExcessiveExchangeIdentifier(long targetSliceSize) {
     this.targetSliceSize = targetSliceSize;
@@ -83,7 +83,7 @@ public class ExcessiveExchangeIdentifier extends BasePrelVisitor<Prel, Excessive
   }
 
   @Override
-  public Prel visitCorrelate(CorrelatePrel prel, MajorFragmentStat s) throws RuntimeException {
+  public Prel visitLateral(LateralJoinPrel prel, MajorFragmentStat s) throws RuntimeException {
     List<RelNode> children = Lists.newArrayList();
     s.add(prel);
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/JoinPrelRenameVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/JoinPrelRenameVisitor.java
index dfb4036..d450c56 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/JoinPrelRenameVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/JoinPrelRenameVisitor.java
@@ -20,7 +20,7 @@ package org.apache.drill.exec.planner.physical.visitor;
 import java.util.List;
 
 import org.apache.drill.exec.planner.physical.JoinPrel;
-import org.apache.drill.exec.planner.physical.CorrelatePrel;
+import org.apache.drill.exec.planner.physical.LateralJoinPrel;
 import org.apache.drill.exec.planner.physical.Prel;
 import org.apache.calcite.rel.RelNode;
 
@@ -72,7 +72,7 @@ public class JoinPrelRenameVisitor extends BasePrelVisitor<Prel, Void, RuntimeEx
 
   //TODO: consolidate this code with join column renaming.
   @Override
-  public Prel visitCorrelate(CorrelatePrel prel, Void value) throws RuntimeException {
+  public Prel visitLateral(LateralJoinPrel prel, Void value) throws RuntimeException {
 
     List<RelNode> children = getChildren(prel);
 
@@ -80,8 +80,8 @@ public class JoinPrelRenameVisitor extends BasePrelVisitor<Prel, Void, RuntimeEx
 
     List<RelNode> reNamedChildren = Lists.newArrayList();
 
-    RelNode left = prel.getCorrelateInput(0, children.get(0));
-    RelNode right = prel.getCorrelateInput(leftCount, children.get(1));
+    RelNode left = prel.getLateralInput(0, children.get(0));
+    RelNode right = prel.getLateralInput(leftCount, children.get(1));
 
     reNamedChildren.add(left);
     reNamedChildren.add(right);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/PrelVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/PrelVisitor.java
index 0e7bbf6..b28d827 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/PrelVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/PrelVisitor.java
@@ -25,20 +25,20 @@ import org.apache.drill.exec.planner.physical.ScanPrel;
 import org.apache.drill.exec.planner.physical.ScreenPrel;
 import org.apache.drill.exec.planner.physical.WriterPrel;
 import org.apache.drill.exec.planner.physical.UnnestPrel;
-import org.apache.drill.exec.planner.physical.CorrelatePrel;
+import org.apache.drill.exec.planner.physical.LateralJoinPrel;
 
 
 public interface PrelVisitor<RETURN, EXTRA, EXCEP extends Throwable> {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PrelVisitor.class);
+  org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PrelVisitor.class);
 
-  public RETURN visitExchange(ExchangePrel prel, EXTRA value) throws EXCEP;
-  public RETURN visitScreen(ScreenPrel prel, EXTRA value) throws EXCEP;
-  public RETURN visitWriter(WriterPrel prel, EXTRA value) throws EXCEP;
-  public RETURN visitScan(ScanPrel prel, EXTRA value) throws EXCEP;
-  public RETURN visitJoin(JoinPrel prel, EXTRA value) throws EXCEP;
-  public RETURN visitProject(ProjectPrel prel, EXTRA value) throws EXCEP;
-  public RETURN visitPrel(Prel prel, EXTRA value) throws EXCEP;
-  public RETURN visitUnnest(UnnestPrel prel, EXTRA value) throws EXCEP;
-  public RETURN visitCorrelate(CorrelatePrel prel, EXTRA value) throws EXCEP;
+  RETURN visitExchange(ExchangePrel prel, EXTRA value) throws EXCEP;
+  RETURN visitScreen(ScreenPrel prel, EXTRA value) throws EXCEP;
+  RETURN visitWriter(WriterPrel prel, EXTRA value) throws EXCEP;
+  RETURN visitScan(ScanPrel prel, EXTRA value) throws EXCEP;
+  RETURN visitJoin(JoinPrel prel, EXTRA value) throws EXCEP;
+  RETURN visitProject(ProjectPrel prel, EXTRA value) throws EXCEP;
+  RETURN visitPrel(Prel prel, EXTRA value) throws EXCEP;
+  RETURN visitUnnest(UnnestPrel prel, EXTRA value) throws EXCEP;
+  RETURN visitLateral(LateralJoinPrel prel, EXTRA value) throws EXCEP;
 
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/PrelVisualizerVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/PrelVisualizerVisitor.java
index 253325b..0bef3a7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/PrelVisualizerVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/PrelVisualizerVisitor.java
@@ -25,7 +25,7 @@ import org.apache.drill.exec.planner.physical.ScanPrel;
 import org.apache.drill.exec.planner.physical.ScreenPrel;
 import org.apache.drill.exec.planner.physical.WriterPrel;
 import org.apache.drill.exec.planner.physical.UnnestPrel;
-import org.apache.drill.exec.planner.physical.CorrelatePrel;
+import org.apache.drill.exec.planner.physical.LateralJoinPrel;
 
 /**
  * Debug-time class that prints a PRel tree to the console for
@@ -234,7 +234,7 @@ public class PrelVisualizerVisitor
   }
 
   @Override
-  public Void visitCorrelate(CorrelatePrel prel, VisualizationState value) throws Exception {
+  public Void visitLateral(LateralJoinPrel prel, VisualizationState value) throws Exception {
     visitPrel(prel, value);
     return null;
   }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/lateraljoin/TestLateralPlans.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/lateraljoin/TestLateralPlans.java
index d027e77..53df9eb 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/lateraljoin/TestLateralPlans.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/lateraljoin/TestLateralPlans.java
@@ -79,7 +79,7 @@ public class TestLateralPlans extends BaseTestQuery {
     String query = "select t.c_name, t2.ord.o_shop as o_shop from cp.`lateraljoin/nested-customer.json` t,"
         + " unnest(t.orders) t2(ord) where t.c_name='customer1' AND t2.ord.o_shop='Meno Park 1st' ";
 
-    PlanTestBase.testPlanMatchingPatterns(query, new String[]{"Correlate(.*[\n\r])+.*Filter(.*[\n\r])+.*Scan(.*[\n\r])+.*Filter"},
+    PlanTestBase.testPlanMatchingPatterns(query, new String[]{"LateralJoin(.*[\n\r])+.*Filter(.*[\n\r])+.*Scan(.*[\n\r])+.*Filter"},
         new String[]{});
 
     testBuilder()
@@ -430,7 +430,7 @@ public class TestLateralPlans extends BaseTestQuery {
   }
 
   private String getRightChildOfLateral(String explain) throws Exception {
-    Matcher matcher = Pattern.compile("Correlate.*Unnest", Pattern.MULTILINE | Pattern.DOTALL).matcher(explain);
+    Matcher matcher = Pattern.compile("LateralJoin.*Unnest", Pattern.MULTILINE | Pattern.DOTALL).matcher(explain);
     assertTrue (matcher.find());
     String CorrelateUnnest = matcher.group(0);
     return CorrelateUnnest.substring(CorrelateUnnest.lastIndexOf("Scan"));
@@ -452,7 +452,7 @@ public class TestLateralPlans extends BaseTestQuery {
       assertTrue(srcOp != null && srcOp.length() > 0);
       String correlateFragmentPattern = srcOp.substring(srcOp.indexOf("=")+1, srcOp.indexOf("]"));
       assertTrue(correlateFragmentPattern != null && correlateFragmentPattern.length() > 0);
-      Matcher matcher = Pattern.compile(correlateFragmentPattern + ".*Correlate", Pattern.MULTILINE | Pattern.DOTALL).matcher(explain);
+      Matcher matcher = Pattern.compile(correlateFragmentPattern + ".*LateralJoin", Pattern.MULTILINE | Pattern.DOTALL).matcher(explain);
       assertTrue(matcher.find());
     }
   }