You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jd...@apache.org on 2018/01/17 21:08:17 UTC

hive git commit: HIVE-18430: Add new determinism category for runtime constants (current_date, current_timestamp) (Jason Dere, reviewed by Jesus Camacho Rodriguez)

Repository: hive
Updated Branches:
  refs/heads/master 790976907 -> 01816fca2


HIVE-18430: Add new determinism category for runtime constants (current_date, current_timestamp) (Jason Dere, reviewed by Jesus Camacho Rodriguez)


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

Branch: refs/heads/master
Commit: 01816fca2c2620f4f57a2b2a5c6ea404e1f0038f
Parents: 7909769
Author: Jason Dere <jd...@hortonworks.com>
Authored: Wed Jan 17 13:07:35 2018 -0800
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Wed Jan 17 13:07:35 2018 -0800

----------------------------------------------------------------------
 .../hadoop/hive/ql/exec/ExprNodeEvaluator.java  | 16 +++++++++
 .../hive/ql/exec/ExprNodeEvaluatorFactory.java  |  2 +-
 .../ql/exec/ExprNodeGenericFuncEvaluator.java   | 12 ++++++-
 .../hadoop/hive/ql/exec/FunctionRegistry.java   | 34 ++++++++++++++++++
 .../optimizer/ConstantPropagateProcFactory.java | 18 +++++-----
 .../PrunerExpressionOperatorFactory.java        |  2 +-
 .../calcite/rules/HiveExceptRewriteRule.java    | 16 ++++-----
 .../calcite/rules/HiveIntersectRewriteRule.java |  2 +-
 .../optimizer/calcite/rules/PartitionPrune.java |  2 +-
 .../translator/SqlFunctionConverter.java        | 23 +++++++++---
 .../ql/optimizer/pcr/PcrExprProcFactory.java    |  2 +-
 .../hive/ql/optimizer/ppr/PartitionPruner.java  |  2 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |  6 ++--
 .../hive/ql/parse/TypeCheckProcFactory.java     |  2 +-
 .../hadoop/hive/ql/plan/ExprNodeDescUtils.java  |  7 ++--
 .../hive/ql/ppd/ExprWalkerProcFactory.java      |  2 +-
 .../apache/hadoop/hive/ql/stats/StatsUtils.java |  2 +-
 .../org/apache/hadoop/hive/ql/udf/UDFType.java  | 10 ++++++
 .../hadoop/hive/ql/udf/generic/GenericUDF.java  |  3 +-
 .../ql/udf/generic/GenericUDFCurrentDate.java   |  6 ++--
 .../udf/generic/GenericUDFCurrentTimestamp.java |  6 ++--
 .../ql/udf/generic/GenericUDFCurrentUser.java   |  4 ++-
 .../ql/udf/generic/GenericUDFLoggedInUser.java  |  4 ++-
 .../hive/ql/udf/generic/GenericUDFMacro.java    |  7 ++++
 .../hive/ql/udf/generic/UDFCurrentDB.java       |  5 ++-
 .../hive/ql/exec/TestFunctionRegistry.java      | 37 ++++++++++++++++++++
 26 files changed, 182 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/01816fca/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeEvaluator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeEvaluator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeEvaluator.java
index 375d65f..f8248a8 100755
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeEvaluator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeEvaluator.java
@@ -102,6 +102,22 @@ public abstract class ExprNodeEvaluator<T extends ExprNodeDesc> {
   }
 
   /**
+   * Return whether this node (or any children nodes) are runtime constants.
+   */
+  public boolean isRuntimeConstant() {
+    return false;
+  }
+
+  /**
+   * Returns whether the expression, for a single query, returns the same result given
+   * the same arguments. This includes deterministic functions as well as runtime
+   * constants (which may not be deterministic across queries).
+   */
+  public boolean isConsistentWithinQuery() {
+    return (isDeterministic() || isRuntimeConstant()) && !isStateful();
+  }
+
+  /**
    * Return child evaluators if exist
    */
   public ExprNodeEvaluator[] getChildren() {

http://git-wip-us.apache.org/repos/asf/hive/blob/01816fca/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeEvaluatorFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeEvaluatorFactory.java
index cc40cae..9c0900b 100755
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeEvaluatorFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeEvaluatorFactory.java
@@ -99,7 +99,7 @@ public final class ExprNodeEvaluatorFactory {
   }
 
   private static ExprNodeEvaluator iterate(ExprNodeEvaluator eval, EvaluatorContext context) {
-    if (!(eval instanceof ExprNodeConstantEvaluator) && eval.isDeterministic()) {
+    if (!(eval instanceof ExprNodeConstantEvaluator) && eval.isConsistentWithinQuery()) {
       ExprNodeEvaluator replace = context.getEvaluated(eval);
       if (replace != null) {
         return replace;

http://git-wip-us.apache.org/repos/asf/hive/blob/01816fca/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeGenericFuncEvaluator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeGenericFuncEvaluator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeGenericFuncEvaluator.java
index 8b9baa6..f119dfe 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeGenericFuncEvaluator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeGenericFuncEvaluator.java
@@ -145,7 +145,7 @@ public class ExprNodeGenericFuncEvaluator extends ExprNodeEvaluator<ExprNodeGene
     }
     outputOI = genericUDF.initializeAndFoldConstants(childrenOIs);
     isConstant = ObjectInspectorUtils.isConstantObjectInspector(outputOI)
-        && isDeterministic();
+        && isConsistentWithinQuery();
     return outputOI;
   }
 
@@ -174,6 +174,16 @@ public class ExprNodeGenericFuncEvaluator extends ExprNodeEvaluator<ExprNodeGene
     return result;
   }
 
+  public boolean isRuntimeConstant() {
+    boolean result = FunctionRegistry.isRuntimeConstant(genericUDF);
+    for (ExprNodeEvaluator child : children) {
+      if(result = result || child.isRuntimeConstant()) {
+        return result;
+      }
+    }
+    return result;
+  }
+
   @Override
   protected Object _evaluate(Object row, int version) throws HiveException {
     if (isConstant) {

http://git-wip-us.apache.org/repos/asf/hive/blob/01816fca/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 68e6ae3..838f078 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
@@ -1492,6 +1492,40 @@ public final class FunctionRegistry {
   }
 
   /**
+   * Returns whether a GenericUDF is a runtime constant or not.
+   */
+  public static boolean isRuntimeConstant(GenericUDF genericUDF) {
+    UDFType genericUDFType = AnnotationUtils.getAnnotation(genericUDF.getClass(), UDFType.class);
+    if (genericUDFType != null && genericUDFType.runtimeConstant()) {
+      return true;
+    }
+
+    if (genericUDF instanceof GenericUDFBridge) {
+      GenericUDFBridge bridge = (GenericUDFBridge) genericUDF;
+      UDFType bridgeUDFType = AnnotationUtils.getAnnotation(bridge.getUdfClass(), UDFType.class);
+      if (bridgeUDFType != null && bridgeUDFType.runtimeConstant()) {
+        return true;
+      }
+    }
+
+    if (genericUDF instanceof GenericUDFMacro) {
+      GenericUDFMacro macro = (GenericUDFMacro) (genericUDF);
+      return macro.isRuntimeConstant();
+    }
+
+    return false;
+  }
+
+  /**
+   * Returns whether the expression, for a single query, returns the same result given
+   * the same arguments/children. This includes deterministic functions as well as runtime
+   * constants (which may not be deterministic across queries).
+   */
+  public static boolean isConsistentWithinQuery(GenericUDF genericUDF) {
+    return (isDeterministic(genericUDF) || isRuntimeConstant(genericUDF)) && !isStateful(genericUDF);
+  }
+
+  /**
    * Returns whether the exprNodeDesc is a node of "and", "or", "not".
    */
   public static boolean isOpAndOrNot(ExprNodeDesc desc) {

http://git-wip-us.apache.org/repos/asf/hive/blob/01816fca/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
index b0a2da8..170e0f6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
@@ -237,7 +237,7 @@ public final class ConstantPropagateProcFactory {
   public static ExprNodeDesc foldExpr(ExprNodeGenericFuncDesc funcDesc) {
 
     GenericUDF udf = funcDesc.getGenericUDF();
-    if (!isDeterministicUdf(udf, funcDesc.getChildren())) {
+    if (!isConstantFoldableUdf(udf, funcDesc.getChildren())) {
       return funcDesc;
     }
     return evaluateFunction(funcDesc.getGenericUDF(),funcDesc.getChildren(), funcDesc.getChildren());
@@ -355,7 +355,7 @@ public final class ConstantPropagateProcFactory {
       }
 
       // Don't evaluate nondeterministic function since the value can only calculate during runtime.
-      if (!isDeterministicUdf(udf, newExprs)) {
+      if (!isConstantFoldableUdf(udf, newExprs)) {
         if (LOG.isDebugEnabled()) {
           LOG.debug("Function " + udf.getClass() + " is undeterministic. Don't evaluate immediately.");
         }
@@ -414,7 +414,7 @@ public final class ConstantPropagateProcFactory {
       }
 
       // Don't evaluate nondeterministic function since the value can only calculate during runtime.
-      if (!isDeterministicUdf(udf, newExprs)) {
+      if (!isConstantFoldableUdf(udf, newExprs)) {
         if (LOG.isDebugEnabled()) {
           LOG.debug("Function " + udf.getClass() + " is undeterministic. Don't evaluate immediately.");
         }
@@ -465,12 +465,12 @@ public final class ConstantPropagateProcFactory {
     return desc;
   }
 
-  private static boolean isDeterministicUdf(GenericUDF udf,  List<ExprNodeDesc> children) {
-    UDFType udfType = udf.getClass().getAnnotation(UDFType.class);
-    if (udf instanceof GenericUDFBridge) {
-      udfType = ((GenericUDFBridge) udf).getUdfClass().getAnnotation(UDFType.class);
-    }
-    if (udfType.deterministic() == false) {
+  /**
+   * Can the UDF be used for constant folding.
+   */
+  private static boolean isConstantFoldableUdf(GenericUDF udf,  List<ExprNodeDesc> children) {
+    // Runtime constants + deterministic functions can be folded.
+    if (!FunctionRegistry.isConsistentWithinQuery(udf)) {
       if (udf.getClass().equals(GenericUDFUnixTimeStamp.class) 
           && children != null && children.size() > 0) {
         // unix_timestamp is polymorphic (ignore class annotations)

http://git-wip-us.apache.org/repos/asf/hive/blob/01816fca/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PrunerExpressionOperatorFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PrunerExpressionOperatorFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PrunerExpressionOperatorFactory.java
index 306e714..a63dc17 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PrunerExpressionOperatorFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PrunerExpressionOperatorFactory.java
@@ -73,7 +73,7 @@ public abstract class PrunerExpressionOperatorFactory {
           }
         }
         unknown = isAllNull;
-      } else if (!FunctionRegistry.isDeterministic(fd.getGenericUDF())) {
+      } else if (!FunctionRegistry.isConsistentWithinQuery(fd.getGenericUDF())) {
         // If it's a non-deterministic UDF, set unknown to true
         unknown = true;
       } else {

http://git-wip-us.apache.org/repos/asf/hive/blob/01816fca/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveExceptRewriteRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveExceptRewriteRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveExceptRewriteRule.java
index 5b72dbd..06ba8c4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveExceptRewriteRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveExceptRewriteRule.java
@@ -275,7 +275,7 @@ public class HiveExceptRewriteRule extends RelOptRule {
         cluster.getTypeFactory()));
     return rexBuilder.makeCall(
         SqlFunctionConverter.getCalciteFn("*", calciteArgTypesBldr.build(),
-            TypeConverter.convert(TypeInfoFactory.longTypeInfo, cluster.getTypeFactory()), true),
+            TypeConverter.convert(TypeInfoFactory.longTypeInfo, cluster.getTypeFactory()), true, false),
         childRexNodeLst);
   }
 
@@ -294,7 +294,7 @@ public class HiveExceptRewriteRule extends RelOptRule {
     // a>0
     RexNode aMorethanZero = rexBuilder.makeCall(
         SqlFunctionConverter.getCalciteFn(">", calciteArgTypesBldr.build(),
-            TypeConverter.convert(TypeInfoFactory.longTypeInfo, cluster.getTypeFactory()), false),
+            TypeConverter.convert(TypeInfoFactory.longTypeInfo, cluster.getTypeFactory()), false, false),
         childRexNodeLst);
     childRexNodeLst = new ArrayList<RexNode>();
     RexLiteral two = rexBuilder.makeBigintLiteral(new BigDecimal(2));
@@ -303,7 +303,7 @@ public class HiveExceptRewriteRule extends RelOptRule {
     // 2*a
     RexNode twoa = rexBuilder.makeCall(
         SqlFunctionConverter.getCalciteFn("*", calciteArgTypesBldr.build(),
-            TypeConverter.convert(TypeInfoFactory.longTypeInfo, cluster.getTypeFactory()), false),
+            TypeConverter.convert(TypeInfoFactory.longTypeInfo, cluster.getTypeFactory()), false, false),
         childRexNodeLst);
     childRexNodeLst = new ArrayList<RexNode>();
     RexInputRef b = rexBuilder.makeInputRef(input, columnSize - 1);
@@ -312,7 +312,7 @@ public class HiveExceptRewriteRule extends RelOptRule {
     // 2a=b
     RexNode twoaEqualTob = rexBuilder.makeCall(
         SqlFunctionConverter.getCalciteFn("=", calciteArgTypesBldr.build(),
-            TypeConverter.convert(TypeInfoFactory.longTypeInfo, cluster.getTypeFactory()), false),
+            TypeConverter.convert(TypeInfoFactory.longTypeInfo, cluster.getTypeFactory()), false, false),
         childRexNodeLst);
     childRexNodeLst = new ArrayList<RexNode>();
     childRexNodeLst.add(aMorethanZero);
@@ -320,7 +320,7 @@ public class HiveExceptRewriteRule extends RelOptRule {
     // a>0 && 2a=b
     return rexBuilder.makeCall(
         SqlFunctionConverter.getCalciteFn("and", calciteArgTypesBldr.build(),
-            TypeConverter.convert(TypeInfoFactory.longTypeInfo, cluster.getTypeFactory()), false),
+            TypeConverter.convert(TypeInfoFactory.longTypeInfo, cluster.getTypeFactory()), false, false),
         childRexNodeLst);
   }
 
@@ -338,7 +338,7 @@ public class HiveExceptRewriteRule extends RelOptRule {
     childRexNodeLst.add(a);
     RexNode threea = rexBuilder.makeCall(
         SqlFunctionConverter.getCalciteFn("*", calciteArgTypesBldr.build(),
-            TypeConverter.convert(TypeInfoFactory.longTypeInfo, cluster.getTypeFactory()), false),
+            TypeConverter.convert(TypeInfoFactory.longTypeInfo, cluster.getTypeFactory()), false, false),
         childRexNodeLst);
 
     RexLiteral two = rexBuilder.makeBigintLiteral(new BigDecimal(2));
@@ -350,7 +350,7 @@ public class HiveExceptRewriteRule extends RelOptRule {
     childRexNodeLst.add(b);
     RexNode twob = rexBuilder.makeCall(
         SqlFunctionConverter.getCalciteFn("*", calciteArgTypesBldr.build(),
-            TypeConverter.convert(TypeInfoFactory.longTypeInfo, cluster.getTypeFactory()), false),
+            TypeConverter.convert(TypeInfoFactory.longTypeInfo, cluster.getTypeFactory()), false, false),
         childRexNodeLst);
 
     // 2b-3a
@@ -359,7 +359,7 @@ public class HiveExceptRewriteRule extends RelOptRule {
     childRexNodeLst.add(threea);
     return rexBuilder.makeCall(
         SqlFunctionConverter.getCalciteFn("-", calciteArgTypesBldr.build(),
-            TypeConverter.convert(TypeInfoFactory.longTypeInfo, cluster.getTypeFactory()), false),
+            TypeConverter.convert(TypeInfoFactory.longTypeInfo, cluster.getTypeFactory()), false, false),
         childRexNodeLst);
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/01816fca/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveIntersectRewriteRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveIntersectRewriteRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveIntersectRewriteRule.java
index a5d950a..c78ac3a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveIntersectRewriteRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveIntersectRewriteRule.java
@@ -183,7 +183,7 @@ public class HiveIntersectRewriteRule extends RelOptRule {
           .makeCall(
               SqlFunctionConverter.getCalciteFn("=", calciteArgTypesBldr.build(),
                   TypeConverter.convert(TypeInfoFactory.longTypeInfo, cluster.getTypeFactory()),
-                  true), childRexNodeLst);
+                  true, false), childRexNodeLst);
     } catch (CalciteSemanticException e) {
       LOG.debug(e.toString());
       throw new RuntimeException(e);

http://git-wip-us.apache.org/repos/asf/hive/blob/01816fca/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/PartitionPrune.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/PartitionPrune.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/PartitionPrune.java
index 0e5e2b9..fab9ff4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/PartitionPrune.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/PartitionPrune.java
@@ -112,7 +112,7 @@ public class PartitionPrune {
         hiveUDF = SqlFunctionConverter.getHiveUDF(call.getOperator(),
             call.getType(), call.operands.size());
         if (hiveUDF != null &&
-            !FunctionRegistry.isDeterministic(hiveUDF)) {
+            !FunctionRegistry.isConsistentWithinQuery(hiveUDF)) {
           return null;
         }
       } finally {

http://git-wip-us.apache.org/repos/asf/hive/blob/01816fca/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
index 13ee4e5..3450254 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
@@ -111,7 +111,12 @@ public class SqlFunctionConverter {
       // let's be proper...
       name = FunctionRegistry.getNormalizedFunctionName(funcTextName);
     }
-    return getCalciteFn(name, calciteArgTypes, retType, FunctionRegistry.isDeterministic(hiveUDF));
+
+    // For calcite, isDeterministic just matters for within the query.
+    // isDynamicFunction used to indicate the function is not deterministic between queries.
+    boolean isDeterministic = FunctionRegistry.isConsistentWithinQuery(hiveUDF);
+    boolean isDynamicFunction = FunctionRegistry.isRuntimeConstant(hiveUDF);
+    return getCalciteFn(name, calciteArgTypes, retType, isDeterministic, isDynamicFunction);
   }
 
   public static SqlOperator getCalciteOperator(String funcTextName, GenericUDTF hiveUDTF,
@@ -119,7 +124,7 @@ public class SqlFunctionConverter {
     // We could just do toLowerCase here and let SA qualify it, but
     // let's be proper...
     String name = FunctionRegistry.getNormalizedFunctionName(funcTextName);
-    return getCalciteFn(name, calciteArgTypes, retType, false);
+    return getCalciteFn(name, calciteArgTypes, retType, false, false);
   }
 
   public static GenericUDF getHiveUDF(SqlOperator op, RelDataType dt, int argsLength) {
@@ -444,18 +449,25 @@ public class SqlFunctionConverter {
 
   private static class CalciteSqlFn extends SqlFunction {
     private final boolean deterministic;
+    private final boolean dynamicFunction;
 
     public CalciteSqlFn(String name, SqlKind kind, SqlReturnTypeInference returnTypeInference,
         SqlOperandTypeInference operandTypeInference, SqlOperandTypeChecker operandTypeChecker,
-        SqlFunctionCategory category, boolean deterministic) {
+        SqlFunctionCategory category, boolean deterministic, boolean dynamicFunction) {
       super(name, kind, returnTypeInference, operandTypeInference, operandTypeChecker, category);
       this.deterministic = deterministic;
+      this.dynamicFunction = dynamicFunction;
     }
 
     @Override
     public boolean isDeterministic() {
       return deterministic;
     }
+
+    @Override
+    public boolean isDynamicFunction() {
+      return dynamicFunction;
+    }
   }
 
   private static class CalciteUDFInfo {
@@ -480,7 +492,8 @@ public class SqlFunctionConverter {
   }
 
   public static SqlOperator getCalciteFn(String hiveUdfName,
-      ImmutableList<RelDataType> calciteArgTypes, RelDataType calciteRetType, boolean deterministic)
+      ImmutableList<RelDataType> calciteArgTypes, RelDataType calciteRetType,
+      boolean deterministic, boolean dynamicFunction)
       throws CalciteSemanticException {
 
     if (hiveUdfName != null && hiveUdfName.trim().equals("<=>")) {
@@ -508,7 +521,7 @@ public class SqlFunctionConverter {
         if (null == calciteOp) {
           calciteOp = new CalciteSqlFn(uInf.udfName, SqlKind.OTHER_FUNCTION, uInf.returnTypeInference,
               uInf.operandTypeInference, uInf.operandTypeChecker,
-              SqlFunctionCategory.USER_DEFINED_FUNCTION, deterministic);
+              SqlFunctionCategory.USER_DEFINED_FUNCTION, deterministic, dynamicFunction);
         }
         break;
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/01816fca/ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PcrExprProcFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PcrExprProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PcrExprProcFactory.java
index 461dbe5..4d16cb0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PcrExprProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PcrExprProcFactory.java
@@ -290,7 +290,7 @@ public final class PcrExprProcFactory {
           }
         }
         return new NodeInfoWrapper(WalkState.PART_COL_STRUCT, null, getOutExpr(fd, nodeOutputs));
-      } else if (!FunctionRegistry.isDeterministic(fd.getGenericUDF())) {
+      } else if (!FunctionRegistry.isConsistentWithinQuery(fd.getGenericUDF())) {
         // If it's a non-deterministic UDF, set unknown to true
         return new NodeInfoWrapper(WalkState.UNKNOWN, null, getOutExpr(fd, nodeOutputs));
       } else {

http://git-wip-us.apache.org/repos/asf/hive/blob/01816fca/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartitionPruner.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartitionPruner.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartitionPruner.java
index ac37cc4..7ee435f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartitionPruner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartitionPruner.java
@@ -120,7 +120,7 @@ public class PartitionPruner extends Transform {
 
     // It cannot contain a non-deterministic function
     if ((expr instanceof ExprNodeGenericFuncDesc)
-        && !FunctionRegistry.isDeterministic(((ExprNodeGenericFuncDesc) expr)
+        && !FunctionRegistry.isConsistentWithinQuery(((ExprNodeGenericFuncDesc) expr)
         .getGenericUDF())) {
       return false;
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/01816fca/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index c558356..82a8cda 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -5288,7 +5288,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       ExprNodeDesc inputExpr = genExprNodeDesc(grpbyExpr,
           reduceSinkInputRowResolver);
       ColumnInfo prev = reduceSinkOutputRowResolver.getExpression(grpbyExpr);
-      if (prev != null && isDeterministic(inputExpr)) {
+      if (prev != null && isConsistentWithinQuery(inputExpr)) {
         colExprMap.put(prev.getInternalName(), inputExpr);
         continue;
       }
@@ -5305,9 +5305,9 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     return reduceKeys;
   }
 
-  private boolean isDeterministic(ExprNodeDesc expr) throws SemanticException {
+  private boolean isConsistentWithinQuery(ExprNodeDesc expr) throws SemanticException {
     try {
-      return ExprNodeEvaluatorFactory.get(expr).isDeterministic();
+      return ExprNodeEvaluatorFactory.get(expr).isConsistentWithinQuery();
     } catch (Exception e) {
       throw new SemanticException(e);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/01816fca/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 67ea32c..d4da35f 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
@@ -1136,7 +1136,7 @@ public class TypeCheckProcFactory {
         // If the function is deterministic and the children are constants,
         // we try to fold the expression to remove e.g. cast on constant
         if (ctx.isFoldExpr() && desc instanceof ExprNodeGenericFuncDesc &&
-                FunctionRegistry.isDeterministic(genericUDF) &&
+                FunctionRegistry.isConsistentWithinQuery(genericUDF) &&
                 ExprNodeDescUtils.isAllConstants(children)) {
           ExprNodeDesc constantExpr = ConstantPropagateProcFactory.foldExpr((ExprNodeGenericFuncDesc)desc);
           if (constantExpr != null) {

http://git-wip-us.apache.org/repos/asf/hive/blob/01816fca/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeDescUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeDescUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeDescUtils.java
index 067fbe0..6cb6d3c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeDescUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeDescUtils.java
@@ -74,8 +74,7 @@ public class ExprNodeDescUtils {
     // for ExprNodeGenericFuncDesc, it should be deterministic and stateless
     if (origin instanceof ExprNodeGenericFuncDesc) {
       ExprNodeGenericFuncDesc func = (ExprNodeGenericFuncDesc) origin;
-      if (!FunctionRegistry.isDeterministic(func.getGenericUDF())
-          || FunctionRegistry.isStateful(func.getGenericUDF())) {
+      if (!FunctionRegistry.isConsistentWithinQuery(func.getGenericUDF())) {
         return null;
       }
       List<ExprNodeDesc> children = new ArrayList<ExprNodeDesc>();
@@ -487,7 +486,7 @@ public class ExprNodeDescUtils {
 
   private static ExprNodeConstantDesc foldConstant(ExprNodeGenericFuncDesc func) {
     GenericUDF udf = func.getGenericUDF();
-    if (!FunctionRegistry.isDeterministic(udf) || FunctionRegistry.isStateful(udf)) {
+    if (!FunctionRegistry.isConsistentWithinQuery(udf)) {
       return null;
     }
     try {
@@ -572,7 +571,7 @@ public class ExprNodeDescUtils {
     }
     if (value instanceof ExprNodeGenericFuncDesc) {
       ExprNodeGenericFuncDesc func = (ExprNodeGenericFuncDesc) value;
-      if (!FunctionRegistry.isDeterministic(func.getGenericUDF())) {
+      if (!FunctionRegistry.isConsistentWithinQuery(func.getGenericUDF())) {
         return false;
       }
       for (ExprNodeDesc child : func.getChildren()) {

http://git-wip-us.apache.org/repos/asf/hive/blob/01816fca/ql/src/java/org/apache/hadoop/hive/ql/ppd/ExprWalkerProcFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ppd/ExprWalkerProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/ppd/ExprWalkerProcFactory.java
index d4df1e8..2ea9229 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ppd/ExprWalkerProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ppd/ExprWalkerProcFactory.java
@@ -198,7 +198,7 @@ public final class ExprWalkerProcFactory {
       String alias = null;
       ExprNodeGenericFuncDesc expr = (ExprNodeGenericFuncDesc) nd;
 
-      if (!FunctionRegistry.isDeterministic(expr.getGenericUDF())) {
+      if (!FunctionRegistry.isConsistentWithinQuery(expr.getGenericUDF())) {
         // this GenericUDF can't be pushed down
         ExprInfo exprInfo = ctx.addOrGetExprInfo(expr);
         exprInfo.isCandidate = false;

http://git-wip-us.apache.org/repos/asf/hive/blob/01816fca/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
index e265863..c414e0b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
@@ -1710,7 +1710,7 @@ public class StatsUtils {
   private static long getNDVFor(ExprNodeGenericFuncDesc engfd, long numRows, Statistics parentStats) {
 
     GenericUDF udf = engfd.getGenericUDF();
-    if (!FunctionRegistry.isDeterministic(udf)){
+    if (!FunctionRegistry.isDeterministic(udf) && !FunctionRegistry.isRuntimeConstant(udf)){
       return numRows;
     }
     List<Long> ndvs = Lists.newArrayList();

http://git-wip-us.apache.org/repos/asf/hive/blob/01816fca/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFType.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFType.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFType.java
index ac3ec58..de0a35c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFType.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFType.java
@@ -62,6 +62,16 @@ public @interface UDFType {
   boolean stateful() default false;
 
   /**
+   * Property used to mark functions like current_timestamp, current_date, current_database().
+   * These functions aren't actually deterministic (the values can change between queries),
+   * but the value returned by these functions should be consistent for the life of the query,
+   * so constant folding still applies for these functions.
+   * Queries using these functions should not be eligible for materialized views or query caching.
+   * @return true if the function is a runtime constant
+   */
+  boolean runtimeConstant() default false;
+
+  /**
    * A UDF is considered distinctLike if the UDF can be evaluated on just the
    * distinct values of a column. Examples include min and max UDFs. This
    * information is used by metadata-only optimizer.

http://git-wip-us.apache.org/repos/asf/hive/blob/01816fca/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDF.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDF.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDF.java
index ef8dcf0..7bcb029 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDF.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDF.java
@@ -163,8 +163,7 @@ public abstract class GenericUDF implements Closeable {
 
     if (allConstant &&
         !ObjectInspectorUtils.isConstantObjectInspector(oi) &&
-        FunctionRegistry.isDeterministic(this) &&
-        !FunctionRegistry.isStateful(this) &&
+        FunctionRegistry.isConsistentWithinQuery(this) &&
         ObjectInspectorUtils.supportsConstantObjectInspector(oi)) {
       DeferredObject[] argumentValues =
         new DeferredJavaObject[arguments.length];

http://git-wip-us.apache.org/repos/asf/hive/blob/01816fca/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCurrentDate.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCurrentDate.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCurrentDate.java
index 1f027a2..22c3dff 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCurrentDate.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCurrentDate.java
@@ -29,9 +29,9 @@ import org.apache.hadoop.hive.serde2.io.DateWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 
-// If there is a new UDFType to describe a function that is deterministic within a query
-// but changes value between queries, this function would fall into that category.
-@UDFType(deterministic = true)
+// This function is not a deterministic function, but a runtime constant.
+// The return value is constant within a query but can be different between queries.
+@UDFType(deterministic = false, runtimeConstant = true)
 @Description(name = "current_date",
     value = "_FUNC_() - Returns the current date at the start of query evaluation."
     + " All calls of current_date within the same query return the same value.")

http://git-wip-us.apache.org/repos/asf/hive/blob/01816fca/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCurrentTimestamp.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCurrentTimestamp.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCurrentTimestamp.java
index 2f13a22..087138c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCurrentTimestamp.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCurrentTimestamp.java
@@ -27,9 +27,9 @@ import org.apache.hadoop.hive.serde2.io.TimestampWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 
-// If there is a new UDFType to describe a function that is deterministic within a query
-// but changes value between queries, this function would fall into that category.
-@UDFType(deterministic = true)
+// This function is not a deterministic function, but a runtime constant.
+// The return value is constant within a query but can be different between queries.
+@UDFType(deterministic = false, runtimeConstant = true)
 @Description(name = "current_timestamp",
     value = "_FUNC_() - Returns the current timestamp at the start of query evaluation."
     + " All calls of current_timestamp within the same query return the same value.")

http://git-wip-us.apache.org/repos/asf/hive/blob/01816fca/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCurrentUser.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCurrentUser.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCurrentUser.java
index d97583d..90e5acd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCurrentUser.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCurrentUser.java
@@ -28,7 +28,9 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.io.Text;
 
-@UDFType(deterministic = true)
+// This function is not a deterministic function, but a runtime constant.
+// The return value is constant within a query but can be different between queries.
+@UDFType(deterministic = false, runtimeConstant = true)
 @Description(name = "current_user", value = "_FUNC_() - Returns current user name", extended = "SessionState UserFromAuthenticator")
 @NDV(maxNdv = 1)
 public class GenericUDFCurrentUser extends GenericUDF {

http://git-wip-us.apache.org/repos/asf/hive/blob/01816fca/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFLoggedInUser.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFLoggedInUser.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFLoggedInUser.java
index 2915b86..5a01566 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFLoggedInUser.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFLoggedInUser.java
@@ -28,7 +28,9 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.io.Text;
 
-@UDFType(deterministic = true)
+// This function is not a deterministic function, but a runtime constant.
+// The return value is constant within a query but can be different between queries.
+@UDFType(deterministic = false, runtimeConstant = true)
 @Description(name = "logged_in_user", value = "_FUNC_() - Returns logged in user name",
         extended = "SessionState GetUserName - the username provided at session initialization")
 @NDV(maxNdv = 1)

http://git-wip-us.apache.org/repos/asf/hive/blob/01816fca/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFMacro.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFMacro.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFMacro.java
index 3f505f2..771ca9d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFMacro.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFMacro.java
@@ -79,6 +79,13 @@ public class GenericUDFMacro extends GenericUDF implements Serializable {
     return false;
   }
  
+  public boolean isRuntimeConstant() {
+    if(body != null) {
+      return body.isRuntimeConstant();
+    }
+    return false;
+  }
+
   private void checkNotNull(Object object, String msg) {
     if(object == null) {
       throw new NullPointerException(msg);

http://git-wip-us.apache.org/repos/asf/hive/blob/01816fca/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/UDFCurrentDB.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/UDFCurrentDB.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/UDFCurrentDB.java
index a5bab4f..1611795 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/UDFCurrentDB.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/UDFCurrentDB.java
@@ -23,12 +23,15 @@ import org.apache.hadoop.hive.ql.exec.MapredContext;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.hive.ql.udf.UDFType;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.io.Text;
 
-// deterministic in the query range
+// This function is not a deterministic function, but a runtime constant.
+// The return value is constant within a query but can be different between queries.
+@UDFType(deterministic = false, runtimeConstant = true)
 @Description(name = "current_database",
     value = "_FUNC_() - returns currently using database name")
 @NDV(maxNdv = 1)

http://git-wip-us.apache.org/repos/asf/hive/blob/01816fca/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFunctionRegistry.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFunctionRegistry.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFunctionRegistry.java
index 3589938..9aa8a16 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFunctionRegistry.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFunctionRegistry.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hive.ql.exec.FunctionInfo.FunctionType;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
 import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.hive.ql.udf.UDFAscii;
 import org.apache.hadoop.hive.ql.udf.UDFLn;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFMax;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
@@ -508,4 +509,40 @@ public class TestFunctionRegistry extends TestCase {
 
     assertTrue("Function should now be recognized as permanent function", FunctionRegistry.isPermanentFunction(fnExpr));
   }
+
+  private GenericUDF getUDF(String udfName) throws Exception {
+    return FunctionRegistry.getFunctionInfo(udfName).getGenericUDF();
+  }
+
+  private void checkRuntimeConstant(GenericUDF udf) {
+    assertFalse(FunctionRegistry.isDeterministic(udf));
+    assertTrue(FunctionRegistry.isRuntimeConstant(udf));
+    assertTrue(FunctionRegistry.isConsistentWithinQuery(udf));
+  }
+
+  private void checkDeterministicFn(GenericUDF udf) {
+    assertTrue(FunctionRegistry.isDeterministic(udf));
+    assertFalse(FunctionRegistry.isRuntimeConstant(udf));
+    assertTrue(FunctionRegistry.isConsistentWithinQuery(udf));
+  }
+
+  private void checkNondeterministicFn(GenericUDF udf) {
+    assertFalse(FunctionRegistry.isDeterministic(udf));
+    assertFalse(FunctionRegistry.isRuntimeConstant(udf));
+    assertFalse(FunctionRegistry.isConsistentWithinQuery(udf));
+  }
+
+  public void testDeterminism() throws Exception {
+    checkDeterministicFn(getUDF("+"));
+    checkDeterministicFn(getUDF("ascii"));
+
+    checkNondeterministicFn(getUDF("rand"));
+    checkNondeterministicFn(getUDF("uuid"));
+
+    checkRuntimeConstant(getUDF("current_database"));
+    checkRuntimeConstant(getUDF("current_date"));
+    checkRuntimeConstant(getUDF("current_timestamp"));
+    checkRuntimeConstant(getUDF("current_user"));
+    checkRuntimeConstant(getUDF("logged_in_user"));
+  }
 }