You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by se...@apache.org on 2016/11/22 02:46:36 UTC

[27/35] hive git commit: HIVE-13557: Make interval keyword optional while specifying DAY in interval arithmetic (Zoltan Haindrich reviewed by Pengcheng Xiong)

HIVE-13557: Make interval keyword optional while specifying DAY in interval arithmetic (Zoltan Haindrich reviewed by Pengcheng Xiong)


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

Branch: refs/heads/hive-14535
Commit: e2a6273afc4dc2a98a9ab238753daf268295fd9f
Parents: 678e6ca
Author: Pengcheng Xiong <px...@apache.org>
Authored: Fri Nov 18 16:02:09 2016 -0800
Committer: Pengcheng Xiong <px...@apache.org>
Committed: Fri Nov 18 16:02:09 2016 -0800

----------------------------------------------------------------------
 .../hadoop/hive/ql/exec/FunctionRegistry.java   |   2 +
 .../org/apache/hadoop/hive/ql/parse/HiveLexer.g |  14 +-
 .../hadoop/hive/ql/parse/IdentifiersParser.g    |  18 +-
 .../hive/ql/parse/TypeCheckProcFactory.java     |  82 -----
 .../udf/generic/GenericUDFInternalInterval.java | 322 +++++++++++++++++++
 .../hive/ql/parse/TestParseDriverIntervals.java |  83 +++++
 .../generic/TestGenericUDFInternalInterval.java | 144 +++++++++
 .../test/queries/clientpositive/interval_alt.q  |  36 +++
 .../results/clientpositive/interval_alt.q.out   | 135 ++++++++
 .../clientpositive/llap/vector_interval_1.q.out |   6 +-
 .../llap/vector_interval_arithmetic.q.out       |  17 +-
 .../results/clientpositive/show_functions.q.out |   1 +
 .../clientpositive/vector_interval_1.q.out      |   6 +-
 .../vector_interval_arithmetic.q.out            |  17 +-
 14 files changed, 763 insertions(+), 120 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/e2a6273a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
index 0dbbc1d..83d78bc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
@@ -468,6 +468,8 @@ public final class FunctionRegistry {
     system.registerGenericUDF("unix_timestamp", GenericUDFUnixTimeStamp.class);
     system.registerGenericUDF("to_unix_timestamp", GenericUDFToUnixTimeStamp.class);
 
+    system.registerGenericUDF("internal_interval", GenericUDFInternalInterval.class);
+
     // Generic UDTF's
     system.registerGenericUDTF("explode", GenericUDTFExplode.class);
     system.registerGenericUDTF("replicate_rows", GenericUDTFReplicateRows.class);

http://git-wip-us.apache.org/repos/asf/hive/blob/e2a6273a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
index 4357328..e9ccfd2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
@@ -303,15 +303,15 @@ KW_AUTHORIZATION: 'AUTHORIZATION';
 KW_CONF: 'CONF';
 KW_VALUES: 'VALUES';
 KW_RELOAD: 'RELOAD';
-KW_YEAR: 'YEAR';
+KW_YEAR: 'YEAR' | 'YEARS';
 KW_QUARTER: 'QUARTER';
-KW_MONTH: 'MONTH';
-KW_WEEK: 'WEEK';
-KW_DAY: 'DAY';
+KW_MONTH: 'MONTH' | 'MONTHS';
+KW_WEEK: 'WEEK' | 'WEEKS';
+KW_DAY: 'DAY' | 'DAYS';
 KW_DOW: 'DAYOFWEEK';
-KW_HOUR: 'HOUR';
-KW_MINUTE: 'MINUTE';
-KW_SECOND: 'SECOND';
+KW_HOUR: 'HOUR' | 'HOURS';
+KW_MINUTE: 'MINUTE' | 'MINUTES';
+KW_SECOND: 'SECOND' | 'SECONDS';
 KW_START: 'START';
 KW_TRANSACTION: 'TRANSACTION';
 KW_COMMIT: 'COMMIT';

http://git-wip-us.apache.org/repos/asf/hive/blob/e2a6273a/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
index 89dad86..1868dda 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
@@ -314,11 +314,9 @@ timeQualifiers
 constant
 @init { gParent.pushMsg("constant", state); }
 @after { gParent.popMsg(state); }
-    :
-    Number
+    : Number
     | dateLiteral
     | timestampLiteral
-    | intervalLiteral
     | StringLiteral
     | stringLiteralSequence
     | IntegralLiteral
@@ -361,12 +359,13 @@ timestampLiteral
     KW_CURRENT_TIMESTAMP -> ^(TOK_FUNCTION KW_CURRENT_TIMESTAMP)
     ;
 
-intervalLiteral
+intervalExpression
     :
-    KW_INTERVAL StringLiteral qualifiers=intervalQualifiers ->
-    {
-      adaptor.create(((CommonTree)qualifiers.getTree()).getType(), $StringLiteral.text)
-    }
+    KW_INTERVAL? k1=(StringLiteral|Number) q1=intervalQualifiers ->
+		^(TOK_FUNCTION Identifier["internal_interval"] NumberLiteral[Integer.toString(((CommonTree)q1.getTree()).token.getType())] $k1)
+    |
+    KW_INTERVAL? LPAREN k2=expression RPAREN q2=intervalQualifiers ->
+		^(TOK_FUNCTION Identifier["internal_interval"] NumberLiteral[Integer.toString(((CommonTree)q2.getTree()).token.getType())] $k2)
     ;
 
 intervalQualifiers
@@ -391,6 +390,7 @@ expression
 atomExpression
     :
     (KW_NULL) => KW_NULL -> TOK_NULL
+    | (intervalExpression)=>intervalExpression
     | (constant) => constant
     | castExpression
     | extractExpression
@@ -770,4 +770,4 @@ nonReserved
 sql11ReservedKeywordsUsedAsFunctionName
     :
     KW_BIGINT | KW_BINARY | KW_BOOLEAN | KW_CURRENT_DATE | KW_CURRENT_TIMESTAMP | KW_DATE | KW_DOUBLE | KW_FLOAT | KW_INT | KW_SMALLINT | KW_TIMESTAMP
-    ;
\ No newline at end of file
+    ;

http://git-wip-us.apache.org/repos/asf/hive/blob/e2a6273a/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
index 5e708d3..96dab3b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
@@ -191,15 +191,6 @@ public class TypeCheckProcFactory {
         + HiveParser.KW_FALSE + "%"), tf.getBoolExprProcessor());
     opRules.put(new RuleRegExp("R5", HiveParser.TOK_DATELITERAL + "%|"
         + HiveParser.TOK_TIMESTAMPLITERAL + "%"), tf.getDateTimeExprProcessor());
-    opRules.put(new RuleRegExp("R6",
-        HiveParser.TOK_INTERVAL_YEAR_MONTH_LITERAL + "%|"
-        + HiveParser.TOK_INTERVAL_DAY_TIME_LITERAL + "%|"
-        + HiveParser.TOK_INTERVAL_YEAR_LITERAL + "%|"
-        + HiveParser.TOK_INTERVAL_MONTH_LITERAL + "%|"
-        + HiveParser.TOK_INTERVAL_DAY_LITERAL + "%|"
-        + HiveParser.TOK_INTERVAL_HOUR_LITERAL + "%|"
-        + HiveParser.TOK_INTERVAL_MINUTE_LITERAL + "%|"
-        + HiveParser.TOK_INTERVAL_SECOND_LITERAL + "%"), tf.getIntervalExprProcessor());
     opRules.put(new RuleRegExp("R7", HiveParser.TOK_TABLE_OR_COL + "%"),
         tf.getColumnExprProcessor());
     opRules.put(new RuleRegExp("R8", HiveParser.TOK_SUBQUERY_OP + "%"),
@@ -512,79 +503,6 @@ public class TypeCheckProcFactory {
   }
 
   /**
-   * Processor for interval constants.
-   */
-  public static class IntervalExprProcessor implements NodeProcessor {
-
-    private static final BigDecimal NANOS_PER_SEC_BD = new BigDecimal(DateUtils.NANOS_PER_SEC);
-    @Override
-    public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
-        Object... nodeOutputs) throws SemanticException {
-
-      TypeCheckCtx ctx = (TypeCheckCtx) procCtx;
-      if (ctx.getError() != null) {
-        return null;
-      }
-
-      ExprNodeDesc desc = TypeCheckProcFactory.processGByExpr(nd, procCtx);
-      if (desc != null) {
-        return desc;
-      }
-
-      ASTNode expr = (ASTNode) nd;
-      String intervalString = BaseSemanticAnalyzer.stripQuotes(expr.getText());
-
-      // Get the string value and convert to a Interval value.
-      try {
-        switch (expr.getType()) {
-          case HiveParser.TOK_INTERVAL_YEAR_MONTH_LITERAL:
-            return new ExprNodeConstantDesc(TypeInfoFactory.intervalYearMonthTypeInfo,
-                HiveIntervalYearMonth.valueOf(intervalString));
-          case HiveParser.TOK_INTERVAL_DAY_TIME_LITERAL:
-            return new ExprNodeConstantDesc(TypeInfoFactory.intervalDayTimeTypeInfo,
-                HiveIntervalDayTime.valueOf(intervalString));
-          case HiveParser.TOK_INTERVAL_YEAR_LITERAL:
-            return new ExprNodeConstantDesc(TypeInfoFactory.intervalYearMonthTypeInfo,
-                new HiveIntervalYearMonth(Integer.parseInt(intervalString), 0));
-          case HiveParser.TOK_INTERVAL_MONTH_LITERAL:
-            return new ExprNodeConstantDesc(TypeInfoFactory.intervalYearMonthTypeInfo,
-                new HiveIntervalYearMonth(0, Integer.parseInt(intervalString)));
-          case HiveParser.TOK_INTERVAL_DAY_LITERAL:
-            return new ExprNodeConstantDesc(TypeInfoFactory.intervalDayTimeTypeInfo,
-                new HiveIntervalDayTime(Integer.parseInt(intervalString), 0, 0, 0, 0));
-          case HiveParser.TOK_INTERVAL_HOUR_LITERAL:
-            return new ExprNodeConstantDesc(TypeInfoFactory.intervalDayTimeTypeInfo,
-                new HiveIntervalDayTime(0, Integer.parseInt(intervalString), 0, 0, 0));
-          case HiveParser.TOK_INTERVAL_MINUTE_LITERAL:
-            return new ExprNodeConstantDesc(TypeInfoFactory.intervalDayTimeTypeInfo,
-                new HiveIntervalDayTime(0, 0, Integer.parseInt(intervalString), 0, 0));
-          case HiveParser.TOK_INTERVAL_SECOND_LITERAL:
-            BigDecimal bd = new BigDecimal(intervalString);
-            BigDecimal bdSeconds = new BigDecimal(bd.toBigInteger());
-            BigDecimal bdNanos = bd.subtract(bdSeconds);
-            return new ExprNodeConstantDesc(TypeInfoFactory.intervalDayTimeTypeInfo,
-                new HiveIntervalDayTime(0, 0, 0, bdSeconds.intValueExact(),
-                    bdNanos.multiply(NANOS_PER_SEC_BD).intValue()));
-          default:
-            throw new IllegalArgumentException("Invalid time literal type " + expr.getType());
-        }
-      } catch (Exception err) {
-        throw new SemanticException(
-            "Unable to convert interval literal '" + intervalString + "' to interval value.", err);
-      }
-    }
-  }
-
-  /**
-   * Factory method to get IntervalExprProcessor.
-   *
-   * @return IntervalExprProcessor.
-   */
-  public IntervalExprProcessor getIntervalExprProcessor() {
-    return new IntervalExprProcessor();
-  }
-
-  /**
    * Processor for table columns.
    */
   public static class ColumnExprProcessor implements NodeProcessor {

http://git-wip-us.apache.org/repos/asf/hive/blob/e2a6273a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFInternalInterval.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFInternalInterval.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFInternalInterval.java
new file mode 100644
index 0000000..fcf291a
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFInternalInterval.java
@@ -0,0 +1,322 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.udf.generic;
+
+import java.math.BigDecimal;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
+import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth;
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.serde2.io.HiveIntervalDayTimeWritable;
+import org.apache.hadoop.hive.serde2.io.HiveIntervalYearMonthWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils.PrimitiveGrouping;
+import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.hive.common.util.DateUtils;
+
+/**
+ * GenericUDF Class for support of "INTERVAL (expression) (DAY|YEAR|...)".
+ */
+@Description(name = "internal_interval", value = "_FUNC_(intervalType,intervalArg)",
+  extended = "this method is not designed to be used by directly calling it - it provides internal support for 'INTERVAL (intervalArg) intervalType' constructs")
+
+public class GenericUDFInternalInterval extends GenericUDF {
+
+  private static Map<Integer, IntervalProcessor> processorMap;
+
+  private transient IntervalProcessor processor;
+  private transient PrimitiveObjectInspector inputOI;
+
+  @Override
+  public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException {
+
+    // read operation mode
+    if (!(arguments[0] instanceof ConstantObjectInspector)) {
+      throw new UDFArgumentTypeException(0,
+          getFuncName() + ": may only accept constant as first argument");
+    }
+    Integer operationMode = getConstantIntValue(arguments, 0);
+    if (operationMode == null) {
+      throw new UDFArgumentTypeException(0, "must supply operationmode");
+    }
+
+    processor = getProcessorMap().get(operationMode);
+    if (processor == null) {
+      throw new UDFArgumentTypeException(0,
+          getFuncName() + ": unsupported operationMode: " + operationMode);
+    }
+
+    // check value argument
+    if (arguments[1].getCategory() != Category.PRIMITIVE) {
+      throw new UDFArgumentTypeException(1,
+          "The first argument to " + getFuncName() + " must be primitive");
+    }
+
+    inputOI = (PrimitiveObjectInspector) arguments[1];
+
+    PrimitiveCategory inputCategory = inputOI.getPrimitiveCategory();
+
+    if (!isValidInputCategory(inputCategory)) {
+      throw new UDFArgumentTypeException(1, "The second argument to " + getFuncName()
+          + " must be from the string group or numeric group (except:float/double)");
+    }
+
+    if (arguments[1] instanceof ConstantObjectInspector) {
+      // return value as constant in case arg is constant
+      return PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
+          processor.getTypeInfo(), processor.evaluate(getConstantStringValue(arguments, 1)));
+    } else {
+      return PrimitiveObjectInspectorFactory
+          .getPrimitiveWritableObjectInspector(processor.getTypeInfo());
+    }
+  }
+
+  private boolean isValidInputCategory(PrimitiveCategory cat) throws UDFArgumentTypeException {
+    PrimitiveGrouping inputOIGroup = PrimitiveObjectInspectorUtils.getPrimitiveGrouping(cat);
+
+    if (inputOIGroup == PrimitiveGrouping.STRING_GROUP) {
+      return true;
+    }
+    if (inputOIGroup == PrimitiveGrouping.NUMERIC_GROUP) {
+      switch (cat) {
+      case DOUBLE:
+      case FLOAT:
+        return false;
+      default:
+        return true;
+      }
+    }
+    return false;
+  }
+
+  @Override
+  public Object evaluate(DeferredObject[] arguments) throws HiveException {
+    String argString = PrimitiveObjectInspectorUtils.getString(arguments[1].get(), inputOI);
+    if (argString == null) {
+      return null;
+    }
+    try {
+      return processor.evaluate(argString);
+    } catch (Exception e) {
+      throw new UDFArgumentTypeException(1, "Error parsing interval " + argString + " using:"
+          + processor.getClass().getSimpleName());
+    }
+  }
+
+  private static interface IntervalProcessor {
+
+    Integer getKey();
+
+    PrimitiveTypeInfo getTypeInfo();
+
+    Object evaluate(String arg) throws UDFArgumentException;
+  }
+
+  private static abstract class AbstractDayTimeIntervalProcessor implements IntervalProcessor {
+    private transient HiveIntervalDayTimeWritable intervalResult =
+        new HiveIntervalDayTimeWritable();
+
+    @Override
+    public final PrimitiveTypeInfo getTypeInfo() {
+      return TypeInfoFactory.intervalDayTimeTypeInfo;
+    }
+
+    @Override
+    public final Object evaluate(String arg) throws UDFArgumentException {
+      intervalResult.set(getIntervalDayTime(arg));
+      return intervalResult;
+    }
+
+    abstract protected HiveIntervalDayTime getIntervalDayTime(String arg);
+  }
+
+  private static abstract class AbstractYearMonthIntervalProcessor implements IntervalProcessor {
+    private transient HiveIntervalYearMonthWritable intervalResult =
+        new HiveIntervalYearMonthWritable();
+
+    @Override
+    public final PrimitiveTypeInfo getTypeInfo() {
+      return TypeInfoFactory.intervalYearMonthTypeInfo;
+    }
+
+    @Override
+    public final Object evaluate(String arg) throws UDFArgumentException {
+      intervalResult.set(getIntervalYearMonth(arg));
+      return intervalResult;
+    }
+
+    abstract protected HiveIntervalYearMonth getIntervalYearMonth(String arg);
+  }
+
+  private static class IntervalDayLiteralProcessor extends AbstractDayTimeIntervalProcessor {
+
+    @Override
+    public Integer getKey() {
+      return HiveParser.TOK_INTERVAL_DAY_LITERAL;
+    }
+
+    @Override
+    protected HiveIntervalDayTime getIntervalDayTime(String arg) {
+      return new HiveIntervalDayTime(Integer.parseInt(arg), 0, 0, 0, 0);
+    }
+  }
+
+  private static class IntervalHourLiteralProcessor extends AbstractDayTimeIntervalProcessor {
+    @Override
+    public Integer getKey() {
+      return HiveParser.TOK_INTERVAL_HOUR_LITERAL;
+    }
+
+    @Override
+    protected HiveIntervalDayTime getIntervalDayTime(String arg) {
+      return new HiveIntervalDayTime(0, Integer.parseInt(arg), 0, 0, 0);
+    }
+  }
+
+  private static class IntervalMinuteLiteralProcessor extends AbstractDayTimeIntervalProcessor {
+    @Override
+    public Integer getKey() {
+      return HiveParser.TOK_INTERVAL_MINUTE_LITERAL;
+    }
+
+    @Override
+    protected HiveIntervalDayTime getIntervalDayTime(String arg) {
+      return new HiveIntervalDayTime(0, 0, Integer.parseInt(arg), 0, 0);
+    }
+  }
+
+  private static class IntervalSecondLiteralProcessor extends AbstractDayTimeIntervalProcessor {
+
+    private static final BigDecimal NANOS_PER_SEC_BD = new BigDecimal(DateUtils.NANOS_PER_SEC);
+
+    @Override
+    public Integer getKey() {
+      return HiveParser.TOK_INTERVAL_SECOND_LITERAL;
+    }
+
+    @Override
+    protected HiveIntervalDayTime getIntervalDayTime(String arg) {
+      BigDecimal bd = new BigDecimal(arg);
+      BigDecimal bdSeconds = new BigDecimal(bd.toBigInteger());
+      BigDecimal bdNanos = bd.subtract(bdSeconds);
+      return new HiveIntervalDayTime(0, 0, 0, bdSeconds.intValueExact(),
+          bdNanos.multiply(NANOS_PER_SEC_BD).intValue());
+    }
+  }
+
+  private static class IntervalDayTimeLiteralProcessor extends AbstractDayTimeIntervalProcessor {
+
+    @Override
+    public Integer getKey() {
+      return HiveParser.TOK_INTERVAL_DAY_TIME_LITERAL;
+    }
+
+    @Override
+    protected HiveIntervalDayTime getIntervalDayTime(String arg) {
+      return HiveIntervalDayTime.valueOf(arg);
+    }
+  }
+
+  private static class IntervalYearMonthLiteralProcessor
+      extends AbstractYearMonthIntervalProcessor {
+
+    @Override
+    public Integer getKey() {
+      return HiveParser.TOK_INTERVAL_YEAR_MONTH_LITERAL;
+    }
+
+    @Override
+    protected HiveIntervalYearMonth getIntervalYearMonth(String arg) {
+      return HiveIntervalYearMonth.valueOf(arg);
+    }
+  }
+
+  private static class IntervalYearLiteralProcessor extends AbstractYearMonthIntervalProcessor {
+
+    @Override
+    public Integer getKey() {
+      return HiveParser.TOK_INTERVAL_YEAR_LITERAL;
+    }
+
+    @Override
+    protected HiveIntervalYearMonth getIntervalYearMonth(String arg) {
+      return new HiveIntervalYearMonth(Integer.parseInt(arg), 0);
+    }
+  }
+
+  private static class IntervalMonthLiteralProcessor extends AbstractYearMonthIntervalProcessor {
+
+    @Override
+    public Integer getKey() {
+      return HiveParser.TOK_INTERVAL_MONTH_LITERAL;
+    }
+
+    @Override
+    protected HiveIntervalYearMonth getIntervalYearMonth(String arg) {
+      return new HiveIntervalYearMonth(0, Integer.parseInt(arg));
+    }
+  }
+
+  private static Map<Integer, IntervalProcessor> getProcessorMap() {
+
+    if (processorMap != null) {
+      return processorMap;
+    }
+
+    Map<Integer, IntervalProcessor> ret = new HashMap<>();
+    IntervalProcessor ips[]=new IntervalProcessor[]{
+        new IntervalDayTimeLiteralProcessor(),
+
+        new IntervalDayLiteralProcessor(),
+        new IntervalHourLiteralProcessor(),
+        new IntervalMinuteLiteralProcessor(),
+        new IntervalSecondLiteralProcessor(),
+
+        new IntervalYearMonthLiteralProcessor(),
+
+        new IntervalYearLiteralProcessor(),
+        new IntervalMonthLiteralProcessor(),
+    };
+
+    for (IntervalProcessor ip : ips) {
+      ret.put(ip.getKey(), ip);
+    }
+
+    return processorMap=ret;
+  }
+
+  @Override
+  public String getDisplayString(String[] children) {
+    return String.format("%s(%s)", processor.getClass().getSimpleName(), children[1]);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/e2a6273a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestParseDriverIntervals.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestParseDriverIntervals.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestParseDriverIntervals.java
new file mode 100644
index 0000000..e5db412
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestParseDriverIntervals.java
@@ -0,0 +1,83 @@
+/**
+ * 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.hadoop.hive.ql.parse;
+
+import static org.junit.Assert.assertNotNull;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.hive.ql.lib.Node;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(Parameterized.class)
+public class TestParseDriverIntervals {
+
+  private String query;
+  private ParseDriver parseDriver;
+
+  @Parameters(name = "{0}")
+  public static List<Object[]> getParameters() {
+    List<Object[]> ret = new ArrayList<>();
+    ret.add(new Object[] { "select 1 days" });
+    ret.add(new Object[] { "select (1) days" });
+    ret.add(new Object[] { "select (1) day" });
+    ret.add(new Object[] { "select (1+1) days" });
+    ret.add(new Object[] { "select interval 1 days" });
+    ret.add(new Object[] { "select interval '1' days" });
+    ret.add(new Object[] { "select interval (x) days" });
+    ret.add(new Object[] { "select interval (x+1) days" });
+    ret.add(new Object[] { "select interval (1+x) days" });
+    ret.add(new Object[] { "select interval (1+1) days" });
+    ret.add(new Object[] { "select interval (x+1) days" });
+
+    return ret;
+  }
+
+  public TestParseDriverIntervals(String query) {
+    parseDriver = new ParseDriver();
+    this.query = query;
+  }
+
+  @Test
+  public void parseInterval() throws Exception {
+    ASTNode root = parseDriver.parse(query);
+    assertNotNull("failed: " + query, findFunctionNode(root));
+    System.out.println(root.dump());
+  }
+
+  private ASTNode findFunctionNode(ASTNode n) {
+    if (n.getType() == HiveParser.TOK_FUNCTION) {
+      if ("internal_interval".equals(n.getChild(0).getText())) {
+        return n;
+      }
+    }
+    ArrayList<Node> children = n.getChildren();
+    if (children != null) {
+      for (Node c : children) {
+        ASTNode r = findFunctionNode((ASTNode) c);
+        if (r != null)
+          return r;
+      }
+    }
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/e2a6273a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFInternalInterval.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFInternalInterval.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFInternalInterval.java
new file mode 100644
index 0000000..8166bc0
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFInternalInterval.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.
+ */
+
+package org.apache.hadoop.hive.ql.udf.generic;
+
+import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredJavaObject;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredObject;
+import org.apache.hadoop.hive.serde2.io.ByteWritable;
+import org.apache.hadoop.hive.serde2.io.HiveIntervalDayTimeWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.Text;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestGenericUDFInternalInterval {
+
+  @Test
+  public void testDayInterval() throws Exception {
+    try (GenericUDFInternalInterval udf = new GenericUDFInternalInterval()) {
+
+      ObjectInspector[] inputOIs =
+          {
+              PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
+                  TypeInfoFactory.intTypeInfo,
+                  new IntWritable(HiveParser.TOK_INTERVAL_DAY_LITERAL)),
+              PrimitiveObjectInspectorFactory.writableStringObjectInspector
+          };
+
+      DeferredObject[] args = {
+          new DeferredJavaObject(new ByteWritable((byte) 4)),
+          new DeferredJavaObject(new Text("8"))
+          };
+
+      PrimitiveObjectInspector oi = (PrimitiveObjectInspector) udf.initialize(inputOIs);
+      Assert.assertEquals(TypeInfoFactory.intervalDayTimeTypeInfo, oi.getTypeInfo());
+      HiveIntervalDayTimeWritable res = (HiveIntervalDayTimeWritable) udf.evaluate(args);
+      Assert.assertEquals(8, res.getHiveIntervalDayTime().getDays());
+    }
+  }
+
+  @Test
+  public void testDayIntervalConstant() throws Exception {
+    try (GenericUDFInternalInterval udf = new GenericUDFInternalInterval()) {
+
+      ObjectInspector[] inputOIs = {
+          PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
+              TypeInfoFactory.intTypeInfo, new IntWritable(HiveParser.TOK_INTERVAL_DAY_LITERAL)),
+          PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
+              TypeInfoFactory.intTypeInfo, new IntWritable(3))
+          };
+
+      PrimitiveObjectInspector oi = (PrimitiveObjectInspector) udf.initialize(inputOIs);
+      Assert.assertEquals(TypeInfoFactory.intervalDayTimeTypeInfo, oi.getTypeInfo());
+      ConstantObjectInspector coi = (ConstantObjectInspector) oi;
+      HiveIntervalDayTimeWritable res =
+          (HiveIntervalDayTimeWritable) coi.getWritableConstantValue();
+      Assert.assertEquals(3, res.getHiveIntervalDayTime().getDays());
+    }
+  }
+
+  @Test(expected = UDFArgumentException.class)
+  public void testDoubleArgumentIsNotSupported() throws Exception {
+    try (GenericUDFInternalInterval udf = new GenericUDFInternalInterval()) {
+
+      ObjectInspector[] inputOIs = {
+          PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
+              TypeInfoFactory.intTypeInfo,
+              new IntWritable(HiveParser.TOK_INTERVAL_DAY_LITERAL)),
+          PrimitiveObjectInspectorFactory.writableDoubleObjectInspector,
+          };
+
+      // should detect double
+      udf.initialize(inputOIs);
+    }
+  }
+
+  @Test(expected = UDFArgumentException.class)
+  public void testInvalidString() throws Exception {
+    try (GenericUDFInternalInterval udf = new GenericUDFInternalInterval()) {
+
+      ObjectInspector[] inputOIs = {
+          PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
+              TypeInfoFactory.intTypeInfo,
+              new IntWritable(HiveParser.TOK_INTERVAL_DAY_LITERAL)),
+          PrimitiveObjectInspectorFactory.writableStringObjectInspector,
+          };
+
+      DeferredObject[] args = {
+          new DeferredJavaObject(new ByteWritable((byte) 4)),
+          new DeferredJavaObject(new Text("invalid"))
+          };
+
+      PrimitiveObjectInspector oi = (PrimitiveObjectInspector) udf.initialize(inputOIs);
+      Assert.assertEquals(TypeInfoFactory.intervalDayTimeTypeInfo, oi.getTypeInfo());
+      // should emit an exception
+      udf.evaluate(args);
+    }
+  }
+
+  @Test
+  public void testNullBypass() throws Exception {
+    try (GenericUDFInternalInterval udf = new GenericUDFInternalInterval()) {
+
+      ObjectInspector[] inputOIs = {
+          PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
+              TypeInfoFactory.intTypeInfo,
+              new IntWritable(HiveParser.TOK_INTERVAL_DAY_LITERAL)),
+          PrimitiveObjectInspectorFactory.writableStringObjectInspector
+          };
+      DeferredObject[] args = {
+              new DeferredJavaObject(new ByteWritable((byte) 4)),
+              new DeferredJavaObject(null)
+              };
+
+      PrimitiveObjectInspector oi = (PrimitiveObjectInspector) udf.initialize(inputOIs);
+      Assert.assertEquals(TypeInfoFactory.intervalDayTimeTypeInfo, oi.getTypeInfo());
+      HiveIntervalDayTimeWritable res = (HiveIntervalDayTimeWritable) udf.evaluate(args);
+      Assert.assertEquals(null, res);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/e2a6273a/ql/src/test/queries/clientpositive/interval_alt.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/interval_alt.q b/ql/src/test/queries/clientpositive/interval_alt.q
new file mode 100644
index 0000000..a4fbe9f
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/interval_alt.q
@@ -0,0 +1,36 @@
+
+select
+	(1) second,
+	 2  seconds,
+	(1) minute,
+	 2  minutes,
+	(1) hour,
+	 2  hours,
+	(1) day,
+	 2  days,
+	(1) month,
+	 2  months,
+	(1) year,
+	 2  years;
+
+select date '2012-01-01' + 30 days;
+select date '2012-01-01' - 30 days;
+
+create table t (dt int);
+insert into t values (1),(2);
+
+-- expressions/columnref
+explain
+select
+	date '2012-01-01' +          (-dt*dt) day,
+	date '2012-01-01' - interval (-dt*dt) day,
+	date '2012-01-01' + 1 day + '2' days,
+	date '2012-01-01' + (dt || '-1') year to month
+	from t;
+
+select
+	date '2012-01-01' +          (-dt*dt) day,
+	date '2012-01-01' - interval (-dt*dt) day,
+	date '2012-01-01' + 1 day + '2' days,
+	date '2012-01-01' + (dt || '-1') year to month
+	from t;

http://git-wip-us.apache.org/repos/asf/hive/blob/e2a6273a/ql/src/test/results/clientpositive/interval_alt.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/interval_alt.q.out b/ql/src/test/results/clientpositive/interval_alt.q.out
new file mode 100644
index 0000000..eb4d10b
--- /dev/null
+++ b/ql/src/test/results/clientpositive/interval_alt.q.out
@@ -0,0 +1,135 @@
+PREHOOK: query: select
+	(1) second,
+	 2  seconds,
+	(1) minute,
+	 2  minutes,
+	(1) hour,
+	 2  hours,
+	(1) day,
+	 2  days,
+	(1) month,
+	 2  months,
+	(1) year,
+	 2  years
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+POSTHOOK: query: select
+	(1) second,
+	 2  seconds,
+	(1) minute,
+	 2  minutes,
+	(1) hour,
+	 2  hours,
+	(1) day,
+	 2  days,
+	(1) month,
+	 2  months,
+	(1) year,
+	 2  years
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+0 00:00:01.000000000	0 00:00:02.000000000	0 00:01:00.000000000	0 00:02:00.000000000	0 01:00:00.000000000	0 02:00:00.000000000	1 00:00:00.000000000	2 00:00:00.000000000	0-1	0-2	1-0	2-0
+PREHOOK: query: select date '2012-01-01' + 30 days
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+POSTHOOK: query: select date '2012-01-01' + 30 days
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+2012-01-31 00:00:00
+PREHOOK: query: select date '2012-01-01' - 30 days
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+POSTHOOK: query: select date '2012-01-01' - 30 days
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+2011-12-02 00:00:00
+PREHOOK: query: create table t (dt int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t
+POSTHOOK: query: create table t (dt int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t
+PREHOOK: query: insert into t values (1),(2)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__1
+PREHOOK: Output: default@t
+POSTHOOK: query: insert into t values (1),(2)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__1
+POSTHOOK: Output: default@t
+POSTHOOK: Lineage: t.dt EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+PREHOOK: query: -- expressions/columnref
+explain
+select
+	date '2012-01-01' +          (-dt*dt) day,
+	date '2012-01-01' - interval (-dt*dt) day,
+	date '2012-01-01' + 1 day + '2' days,
+	date '2012-01-01' + (dt || '-1') year to month
+	from t
+PREHOOK: type: QUERY
+POSTHOOK: query: -- expressions/columnref
+explain
+select
+	date '2012-01-01' +          (-dt*dt) day,
+	date '2012-01-01' - interval (-dt*dt) day,
+	date '2012-01-01' + 1 day + '2' days,
+	date '2012-01-01' + (dt || '-1') year to month
+	from t
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: t
+            Statistics: Num rows: 2 Data size: 2 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: (2012-01-01 + IntervalDayLiteralProcessor(((- dt) * dt))) (type: timestamp), (2012-01-01 - IntervalDayLiteralProcessor(((- dt) * dt))) (type: timestamp), 2012-01-04 (type: date), (2012-01-01 + IntervalYearMonthLiteralProcessor(concat(dt, '-1'))) (type: date)
+              outputColumnNames: _col0, _col1, _col2, _col3
+              Statistics: Num rows: 2 Data size: 2 Basic stats: COMPLETE Column stats: NONE
+              File Output Operator
+                compressed: false
+                Statistics: Num rows: 2 Data size: 2 Basic stats: COMPLETE Column stats: NONE
+                table:
+                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select
+	date '2012-01-01' +          (-dt*dt) day,
+	date '2012-01-01' - interval (-dt*dt) day,
+	date '2012-01-01' + 1 day + '2' days,
+	date '2012-01-01' + (dt || '-1') year to month
+	from t
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t
+#### A masked pattern was here ####
+POSTHOOK: query: select
+	date '2012-01-01' +          (-dt*dt) day,
+	date '2012-01-01' - interval (-dt*dt) day,
+	date '2012-01-01' + 1 day + '2' days,
+	date '2012-01-01' + (dt || '-1') year to month
+	from t
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t
+#### A masked pattern was here ####
+2011-12-31 00:00:00	2012-01-02 00:00:00	2012-01-04	2013-02-01
+2011-12-28 00:00:00	2012-01-05 00:00:00	2012-01-04	2014-02-01

http://git-wip-us.apache.org/repos/asf/hive/blob/e2a6273a/ql/src/test/results/clientpositive/llap/vector_interval_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_interval_1.q.out b/ql/src/test/results/clientpositive/llap/vector_interval_1.q.out
index d8003ba..0dd8fc5 100644
--- a/ql/src/test/results/clientpositive/llap/vector_interval_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_interval_1.q.out
@@ -86,7 +86,7 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
-                expressions: KEY.reducesinkkey0 (type: string), 1-2 (type: interval_year_month), VALUE._col0 (type: interval_year_month), 1 02:03:04.000000000 (type: interval_day_time), VALUE._col1 (type: interval_day_time)
+                expressions: KEY.reducesinkkey0 (type: string), 1-2 (type: interval_year_month), VALUE._col1 (type: interval_year_month), 1 02:03:04.000000000 (type: interval_day_time), VALUE._col3 (type: interval_day_time)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4
                 Statistics: Num rows: 2 Data size: 442 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
@@ -177,7 +177,7 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
-                expressions: KEY.reducesinkkey0 (type: date), 2-4 (type: interval_year_month), VALUE._col0 (type: interval_year_month), VALUE._col1 (type: interval_year_month), 0-0 (type: interval_year_month), VALUE._col2 (type: interval_year_month), VALUE._col3 (type: interval_year_month)
+                expressions: KEY.reducesinkkey0 (type: date), 2-4 (type: interval_year_month), VALUE._col1 (type: interval_year_month), VALUE._col2 (type: interval_year_month), 0-0 (type: interval_year_month), VALUE._col4 (type: interval_year_month), VALUE._col5 (type: interval_year_month)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                 Statistics: Num rows: 2 Data size: 442 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
@@ -274,7 +274,7 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
-                expressions: KEY.reducesinkkey0 (type: date), 2 04:06:08.000000000 (type: interval_day_time), VALUE._col1 (type: interval_day_time), VALUE._col2 (type: interval_day_time), 0 00:00:00.000000000 (type: interval_day_time), VALUE._col4 (type: interval_day_time), VALUE._col5 (type: interval_day_time)
+                expressions: KEY.reducesinkkey0 (type: date), 2 04:06:08.000000000 (type: interval_day_time), VALUE._col0 (type: interval_day_time), VALUE._col1 (type: interval_day_time), 0 00:00:00.000000000 (type: interval_day_time), VALUE._col2 (type: interval_day_time), VALUE._col3 (type: interval_day_time)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                 Statistics: Num rows: 2 Data size: 442 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/e2a6273a/ql/src/test/results/clientpositive/llap/vector_interval_arithmetic.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_interval_arithmetic.q.out b/ql/src/test/results/clientpositive/llap/vector_interval_arithmetic.q.out
index 13a8b35..8e3bc39 100644
--- a/ql/src/test/results/clientpositive/llap/vector_interval_arithmetic.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_interval_arithmetic.q.out
@@ -137,7 +137,7 @@ order by dateval
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@interval_arithmetic_1
 #### A masked pattern was here ####
-dateval	c1	c2	c3	c4	c5	c6
+dateval	_c1	_c2	_c3	_c4	_c5	_c6
 0004-09-22	0002-07-22	0006-11-22	0006-11-22	0002-07-22	0002-07-22	0006-11-22
 0528-10-27	0526-08-27	0530-12-27	0530-12-27	0526-08-27	0526-08-27	0530-12-27
 1319-02-02	1316-12-02	1321-04-02	1321-04-02	1316-12-02	1316-12-02	1321-04-02
@@ -427,7 +427,7 @@ order by tsval
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@interval_arithmetic_1
 #### A masked pattern was here ####
-tsval	c1	c2	c3	c4	c5	c6
+tsval	_c1	_c2	_c3	_c4	_c5	_c6
 0004-09-22 18:26:29.519542222	0002-07-22 18:26:29.519542222	0006-11-22 18:26:29.519542222	0006-11-22 18:26:29.519542222	0002-07-22 18:26:29.519542222	0002-07-22 18:26:29.519542222	0006-11-22 18:26:29.519542222
 0528-10-27 08:15:18.941718273	0526-08-27 08:15:18.941718273	0530-12-27 08:15:18.941718273	0530-12-27 08:15:18.941718273	0526-08-27 08:15:18.941718273	0526-08-27 08:15:18.941718273	0530-12-27 08:15:18.941718273
 1319-02-02 16:31:57.778	1316-12-02 16:31:57.778	1321-04-02 16:31:57.778	1321-04-02 16:31:57.778	1316-12-02 16:31:57.778	1316-12-02 16:31:57.778	1321-04-02 16:31:57.778
@@ -515,7 +515,8 @@ STAGE PLANS:
                   Select Operator
                     Statistics: Num rows: 50 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
-                      sort order: 
+                      key expressions: 5-5 (type: interval_year_month)
+                      sort order: +
                       Statistics: Num rows: 50 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
                       TopN Hash Memory Usage: 0.1
             Execution mode: vectorized, llap
@@ -524,7 +525,7 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
-                expressions: 5-5 (type: interval_year_month), -1-1 (type: interval_year_month)
+                expressions: KEY.reducesinkkey0 (type: interval_year_month), -1-1 (type: interval_year_month)
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 50 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
@@ -562,7 +563,7 @@ limit 2
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@interval_arithmetic_1
 #### A masked pattern was here ####
-c0	c1
+_c0	_c1
 5-5	-1-1
 5-5	-1-1
 PREHOOK: query: -- interval day-time arithmetic
@@ -667,7 +668,7 @@ order by dateval
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@interval_arithmetic_1
 #### A masked pattern was here ####
-dateval	_c1	_c2	_c3	_c4	_c5	_c6
+dateval	c1	c2	c3	c4	c5	c6
 0004-09-22	0004-06-14 12:37:26.876543211	0004-12-30 11:22:33.123456789	0004-12-30 11:22:33.123456789	0004-06-14 12:37:26.876543211	0004-06-14 12:37:26.876543211	0004-12-30 11:22:33.123456789
 0528-10-27	0528-07-19 12:37:26.876543211	0529-02-03 11:22:33.123456789	0529-02-03 11:22:33.123456789	0528-07-19 12:37:26.876543211	0528-07-19 12:37:26.876543211	0529-02-03 11:22:33.123456789
 1319-02-02	1318-10-25 12:37:26.876543211	1319-05-12 11:22:33.123456789	1319-05-12 11:22:33.123456789	1318-10-25 12:37:26.876543211	1318-10-25 12:37:26.876543211	1319-05-12 11:22:33.123456789
@@ -961,7 +962,7 @@ order by tsval
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@interval_arithmetic_1
 #### A masked pattern was here ####
-tsval	_c1	_c2	_c3	_c4	_c5	_c6
+tsval	c1	c2	c3	c4	c5	c6
 0004-09-22 18:26:29.519542222	0004-06-15 07:03:56.396085433	0004-12-31 05:49:02.642999011	0004-12-31 05:49:02.642999011	0004-06-15 07:03:56.396085433	0004-06-15 07:03:56.396085433	0004-12-31 05:49:02.642999011
 0528-10-27 08:15:18.941718273	0528-07-19 20:52:45.818261484	0529-02-03 19:37:52.065175062	0529-02-03 19:37:52.065175062	0528-07-19 20:52:45.818261484	0528-07-19 20:52:45.818261484	0529-02-03 19:37:52.065175062
 1319-02-02 16:31:57.778	1318-10-26 05:09:24.654543211	1319-05-13 03:54:30.901456789	1319-05-13 03:54:30.901456789	1318-10-26 05:09:24.654543211	1318-10-26 05:09:24.654543211	1319-05-13 03:54:30.901456789
@@ -1080,7 +1081,7 @@ limit 2
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@interval_arithmetic_1
 #### A masked pattern was here ####
-_c0	_c1
+c0	c1
 109 20:30:40.246913578	89 02:14:26.000000000
 109 20:30:40.246913578	89 02:14:26.000000000
 PREHOOK: query: drop table interval_arithmetic_1

http://git-wip-us.apache.org/repos/asf/hive/blob/e2a6273a/ql/src/test/results/clientpositive/show_functions.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/show_functions.q.out b/ql/src/test/results/clientpositive/show_functions.q.out
index d2f3b97..6cb9015 100644
--- a/ql/src/test/results/clientpositive/show_functions.q.out
+++ b/ql/src/test/results/clientpositive/show_functions.q.out
@@ -113,6 +113,7 @@ index
 initcap
 inline
 instr
+internal_interval
 isnotnull
 isnull
 java_method

http://git-wip-us.apache.org/repos/asf/hive/blob/e2a6273a/ql/src/test/results/clientpositive/vector_interval_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_interval_1.q.out b/ql/src/test/results/clientpositive/vector_interval_1.q.out
index 373a6de..f18e56c 100644
--- a/ql/src/test/results/clientpositive/vector_interval_1.q.out
+++ b/ql/src/test/results/clientpositive/vector_interval_1.q.out
@@ -77,7 +77,7 @@ STAGE PLANS:
       Execution mode: vectorized
       Reduce Operator Tree:
         Select Operator
-          expressions: KEY.reducesinkkey0 (type: string), 1-2 (type: interval_year_month), VALUE._col0 (type: interval_year_month), 1 02:03:04.000000000 (type: interval_day_time), VALUE._col1 (type: interval_day_time)
+          expressions: KEY.reducesinkkey0 (type: string), 1-2 (type: interval_year_month), VALUE._col1 (type: interval_year_month), 1 02:03:04.000000000 (type: interval_day_time), VALUE._col3 (type: interval_day_time)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4
           Statistics: Num rows: 2 Data size: 442 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
@@ -159,7 +159,7 @@ STAGE PLANS:
       Execution mode: vectorized
       Reduce Operator Tree:
         Select Operator
-          expressions: KEY.reducesinkkey0 (type: date), 2-4 (type: interval_year_month), VALUE._col0 (type: interval_year_month), VALUE._col1 (type: interval_year_month), 0-0 (type: interval_year_month), VALUE._col2 (type: interval_year_month), VALUE._col3 (type: interval_year_month)
+          expressions: KEY.reducesinkkey0 (type: date), 2-4 (type: interval_year_month), VALUE._col1 (type: interval_year_month), VALUE._col2 (type: interval_year_month), 0-0 (type: interval_year_month), VALUE._col4 (type: interval_year_month), VALUE._col5 (type: interval_year_month)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
           Statistics: Num rows: 2 Data size: 442 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
@@ -247,7 +247,7 @@ STAGE PLANS:
       Execution mode: vectorized
       Reduce Operator Tree:
         Select Operator
-          expressions: KEY.reducesinkkey0 (type: date), 2 04:06:08.000000000 (type: interval_day_time), VALUE._col1 (type: interval_day_time), VALUE._col2 (type: interval_day_time), 0 00:00:00.000000000 (type: interval_day_time), VALUE._col4 (type: interval_day_time), VALUE._col5 (type: interval_day_time)
+          expressions: KEY.reducesinkkey0 (type: date), 2 04:06:08.000000000 (type: interval_day_time), VALUE._col0 (type: interval_day_time), VALUE._col1 (type: interval_day_time), 0 00:00:00.000000000 (type: interval_day_time), VALUE._col2 (type: interval_day_time), VALUE._col3 (type: interval_day_time)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
           Statistics: Num rows: 2 Data size: 442 Basic stats: COMPLETE Column stats: NONE
           File Output Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/e2a6273a/ql/src/test/results/clientpositive/vector_interval_arithmetic.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_interval_arithmetic.q.out b/ql/src/test/results/clientpositive/vector_interval_arithmetic.q.out
index ff16b3b..5d1dca6 100644
--- a/ql/src/test/results/clientpositive/vector_interval_arithmetic.q.out
+++ b/ql/src/test/results/clientpositive/vector_interval_arithmetic.q.out
@@ -128,7 +128,7 @@ order by dateval
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@interval_arithmetic_1
 #### A masked pattern was here ####
-dateval	c1	c2	c3	c4	c5	c6
+dateval	_c1	_c2	_c3	_c4	_c5	_c6
 0004-09-22	0002-07-22	0006-11-22	0006-11-22	0002-07-22	0002-07-22	0006-11-22
 0528-10-27	0526-08-27	0530-12-27	0530-12-27	0526-08-27	0526-08-27	0530-12-27
 1319-02-02	1316-12-02	1321-04-02	1321-04-02	1316-12-02	1316-12-02	1321-04-02
@@ -400,7 +400,7 @@ order by tsval
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@interval_arithmetic_1
 #### A masked pattern was here ####
-tsval	c1	c2	c3	c4	c5	c6
+tsval	_c1	_c2	_c3	_c4	_c5	_c6
 0004-09-22 18:26:29.519542222	0002-07-22 18:26:29.519542222	0006-11-22 18:26:29.519542222	0006-11-22 18:26:29.519542222	0002-07-22 18:26:29.519542222	0002-07-22 18:26:29.519542222	0006-11-22 18:26:29.519542222
 0528-10-27 08:15:18.941718273	0526-08-27 08:15:18.941718273	0530-12-27 08:15:18.941718273	0530-12-27 08:15:18.941718273	0526-08-27 08:15:18.941718273	0526-08-27 08:15:18.941718273	0530-12-27 08:15:18.941718273
 1319-02-02 16:31:57.778	1316-12-02 16:31:57.778	1321-04-02 16:31:57.778	1321-04-02 16:31:57.778	1316-12-02 16:31:57.778	1316-12-02 16:31:57.778	1321-04-02 16:31:57.778
@@ -482,13 +482,14 @@ STAGE PLANS:
             Select Operator
               Statistics: Num rows: 50 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
               Reduce Output Operator
-                sort order: 
+                key expressions: 5-5 (type: interval_year_month)
+                sort order: +
                 Statistics: Num rows: 50 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
                 TopN Hash Memory Usage: 0.1
       Execution mode: vectorized
       Reduce Operator Tree:
         Select Operator
-          expressions: 5-5 (type: interval_year_month), -1-1 (type: interval_year_month)
+          expressions: KEY.reducesinkkey0 (type: interval_year_month), -1-1 (type: interval_year_month)
           outputColumnNames: _col0, _col1
           Statistics: Num rows: 50 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
           Limit
@@ -526,7 +527,7 @@ limit 2
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@interval_arithmetic_1
 #### A masked pattern was here ####
-c0	c1
+_c0	_c1
 5-5	-1-1
 5-5	-1-1
 PREHOOK: query: -- interval day-time arithmetic
@@ -622,7 +623,7 @@ order by dateval
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@interval_arithmetic_1
 #### A masked pattern was here ####
-dateval	_c1	_c2	_c3	_c4	_c5	_c6
+dateval	c1	c2	c3	c4	c5	c6
 0004-09-22	0004-06-14 12:37:26.876543211	0004-12-30 11:22:33.123456789	0004-12-30 11:22:33.123456789	0004-06-14 12:37:26.876543211	0004-06-14 12:37:26.876543211	0004-12-30 11:22:33.123456789
 0528-10-27	0528-07-19 12:37:26.876543211	0529-02-03 11:22:33.123456789	0529-02-03 11:22:33.123456789	0528-07-19 12:37:26.876543211	0528-07-19 12:37:26.876543211	0529-02-03 11:22:33.123456789
 1319-02-02	1318-10-25 12:37:26.876543211	1319-05-12 11:22:33.123456789	1319-05-12 11:22:33.123456789	1318-10-25 12:37:26.876543211	1318-10-25 12:37:26.876543211	1319-05-12 11:22:33.123456789
@@ -898,7 +899,7 @@ order by tsval
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@interval_arithmetic_1
 #### A masked pattern was here ####
-tsval	_c1	_c2	_c3	_c4	_c5	_c6
+tsval	c1	c2	c3	c4	c5	c6
 0004-09-22 18:26:29.519542222	0004-06-15 07:03:56.396085433	0004-12-31 05:49:02.642999011	0004-12-31 05:49:02.642999011	0004-06-15 07:03:56.396085433	0004-06-15 07:03:56.396085433	0004-12-31 05:49:02.642999011
 0528-10-27 08:15:18.941718273	0528-07-19 20:52:45.818261484	0529-02-03 19:37:52.065175062	0529-02-03 19:37:52.065175062	0528-07-19 20:52:45.818261484	0528-07-19 20:52:45.818261484	0529-02-03 19:37:52.065175062
 1319-02-02 16:31:57.778	1318-10-26 05:09:24.654543211	1319-05-13 03:54:30.901456789	1319-05-13 03:54:30.901456789	1318-10-26 05:09:24.654543211	1318-10-26 05:09:24.654543211	1319-05-13 03:54:30.901456789
@@ -1013,7 +1014,7 @@ limit 2
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@interval_arithmetic_1
 #### A masked pattern was here ####
-_c0	_c1
+c0	c1
 109 20:30:40.246913578	89 02:14:26.000000000
 109 20:30:40.246913578	89 02:14:26.000000000
 PREHOOK: query: drop table interval_arithmetic_1