You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2016/01/21 23:38:40 UTC

[03/50] [abbrv] calcite git commit: [CALCITE-1041] User-defined function that returns DATE or TIMESTAMP value

[CALCITE-1041] User-defined function that returns DATE or TIMESTAMP value

Rationalize code that translates to an from internal representation (e.g. DATE stored as int).

Fix an bug with overloaded UDFs, and a bug with UDF args of type SMALLINT (short).

Deprecate Utilities.equal.

Add methods to in BuiltInMethod.


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

Branch: refs/heads/branch-release
Commit: d4bbf58d6e5f4f67ebae412b5a1aae23769eccb7
Parents: efb668b
Author: Julian Hyde <jh...@apache.org>
Authored: Tue Jan 5 15:15:51 2016 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed Jan 6 02:37:31 2016 -0800

----------------------------------------------------------------------
 .../calcite/adapter/enumerable/EnumUtils.java   | 118 ++++++++++++++++++-
 .../enumerable/EnumerableRelImplementor.java    |  19 +--
 .../adapter/enumerable/PhysTypeImpl.java        |  15 +--
 .../ReflectiveCallNotNullImplementor.java       |  36 +-----
 .../calcite/adapter/enumerable/RexImpTable.java |   7 +-
 .../adapter/enumerable/RexToLixTranslator.java  |  39 ++++--
 .../enumerable/StrictAggImplementor.java        |  22 ++--
 .../calcite/prepare/CalciteCatalogReader.java   |  48 +++++---
 .../org/apache/calcite/runtime/FlatLists.java   |  11 +-
 .../apache/calcite/runtime/SqlFunctions.java    |  37 ++++--
 .../org/apache/calcite/runtime/Utilities.java   |   2 +
 .../java/org/apache/calcite/sql/SqlUtil.java    |   2 +-
 .../sql/type/SqlTypeExplicitPrecedenceList.java |   1 +
 .../apache/calcite/sql/type/SqlTypeUtil.java    |   4 +
 .../org/apache/calcite/util/BuiltInMethod.java  |  16 ++-
 .../org/apache/calcite/test/CalciteAssert.java  |   3 +-
 .../java/org/apache/calcite/test/UdfTest.java   |  62 ++++++++++
 .../java/org/apache/calcite/util/Smalls.java    |  35 ++++--
 18 files changed, 357 insertions(+), 120 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumUtils.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumUtils.java
index 17e0bf5..60408c6 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumUtils.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumUtils.java
@@ -20,6 +20,7 @@ import org.apache.calcite.adapter.java.JavaTypeFactory;
 import org.apache.calcite.linq4j.Ord;
 import org.apache.calcite.linq4j.function.Function2;
 import org.apache.calcite.linq4j.tree.BlockStatement;
+import org.apache.calcite.linq4j.tree.ConstantUntypedNull;
 import org.apache.calcite.linq4j.tree.Expression;
 import org.apache.calcite.linq4j.tree.Expressions;
 import org.apache.calcite.linq4j.tree.MethodDeclaration;
@@ -29,8 +30,11 @@ import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.BuiltInMethod;
 
+import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
 
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
@@ -44,6 +48,14 @@ import java.util.List;
  * style.
  */
 public class EnumUtils {
+
+  private static final Function<RexNode, Type> REX_TO_INTERNAL_TYPE =
+      new Function<RexNode, Type>() {
+        public Type apply(RexNode node) {
+          return toInternal(node.getType());
+        }
+      };
+
   private EnumUtils() {}
 
   static final boolean BRIDGE_METHODS = true;
@@ -54,7 +66,8 @@ public class EnumUtils {
   static final List<Expression> NO_EXPRS =
       ImmutableList.of();
 
-  public static final String[] LEFT_RIGHT = {"left", "right"};
+  public static final List<String> LEFT_RIGHT =
+      ImmutableList.of("left", "right");
 
   /** Declares a method that overrides another method. */
   public static MethodDeclaration overridingMethodDecl(Method method,
@@ -117,12 +130,10 @@ public class EnumUtils {
   static Expression joinSelector(JoinRelType joinType, PhysType physType,
       List<PhysType> inputPhysTypes) {
     // A parameter for each input.
-    final List<ParameterExpression> parameters =
-        new ArrayList<ParameterExpression>();
+    final List<ParameterExpression> parameters = new ArrayList<>();
 
     // Generate all fields.
-    final List<Expression> expressions =
-        new ArrayList<Expression>();
+    final List<Expression> expressions = new ArrayList<>();
     final int outputFieldCount = physType.getRowType().getFieldCount();
     for (Ord<PhysType> ord : Ord.zip(inputPhysTypes)) {
       final PhysType inputPhysType =
@@ -132,7 +143,7 @@ public class EnumUtils {
       // Function<T> always operates on boxed arguments
       final ParameterExpression parameter =
           Expressions.parameter(Primitive.box(inputPhysType.getJavaRowType()),
-              EnumUtils.LEFT_RIGHT[ord.i]);
+              EnumUtils.LEFT_RIGHT.get(ord.i));
       parameters.add(parameter);
       if (expressions.size() == outputFieldCount) {
         // For instance, if semi-join needs to return just the left inputs
@@ -158,6 +169,101 @@ public class EnumUtils {
         physType.record(expressions),
         parameters);
   }
+
+  /** Converts from internal representation to JDBC representation used by
+   * arguments of user-defined functions. For example, converts date values from
+   * {@code int} to {@link java.sql.Date}. */
+  static Expression fromInternal(Expression e, Class<?> targetType) {
+    if (e == ConstantUntypedNull.INSTANCE) {
+      return e;
+    }
+    if (!(e.getType() instanceof Class)) {
+      return e;
+    }
+    if (targetType.isAssignableFrom((Class) e.getType())) {
+      return e;
+    }
+    if (targetType == java.sql.Date.class) {
+      return Expressions.call(BuiltInMethod.INTERNAL_TO_DATE.method, e);
+    }
+    if (targetType == java.sql.Time.class) {
+      return Expressions.call(BuiltInMethod.INTERNAL_TO_TIME.method, e);
+    }
+    if (targetType == java.sql.Timestamp.class) {
+      return Expressions.call(BuiltInMethod.INTERNAL_TO_TIMESTAMP.method, e);
+    }
+    if (Primitive.is(e.type)
+        && Primitive.isBox(targetType)) {
+      // E.g. e is "int", target is "Long", generate "(long) e".
+      return Expressions.convert_(e,
+          Primitive.ofBox(targetType).primitiveClass);
+    }
+    return e;
+  }
+
+  static List<Expression> fromInternal(Class<?>[] targetTypes,
+      List<Expression> expressions) {
+    final List<Expression> list = new ArrayList<>();
+    for (int i = 0; i < expressions.size(); i++) {
+      list.add(fromInternal(expressions.get(i), targetTypes[i]));
+    }
+    return list;
+  }
+
+  static Type fromInternal(Type type) {
+    if (type == java.sql.Date.class || type == java.sql.Time.class) {
+      return int.class;
+    }
+    if (type == java.sql.Timestamp.class) {
+      return long.class;
+    }
+    return type;
+  }
+
+  static Type toInternal(RelDataType type) {
+    switch (type.getSqlTypeName()) {
+    case DATE:
+    case TIME:
+      return type.isNullable() ? Integer.class : int.class;
+    case TIMESTAMP:
+      return type.isNullable() ? Long.class : long.class;
+    default:
+      return null; // we don't care; use the default storage type
+    }
+  }
+
+  static List<Type> internalTypes(List<? extends RexNode> operandList) {
+    return Lists.transform(operandList, REX_TO_INTERNAL_TYPE);
+  }
+
+  static Expression enforce(final Type storageType,
+      final Expression e) {
+    if (storageType != null && e.type != storageType) {
+      if (e.type == java.sql.Date.class) {
+        if (storageType == int.class) {
+          return Expressions.call(BuiltInMethod.DATE_TO_INT.method, e);
+        }
+        if (storageType == Integer.class) {
+          return Expressions.call(BuiltInMethod.DATE_TO_INT_OPTIONAL.method, e);
+        }
+      } else if (e.type == java.sql.Time.class) {
+        if (storageType == int.class) {
+          return Expressions.call(BuiltInMethod.TIME_TO_INT.method, e);
+        }
+        if (storageType == Integer.class) {
+          return Expressions.call(BuiltInMethod.TIME_TO_INT_OPTIONAL.method, e);
+        }
+      } else if (e.type == java.sql.Timestamp.class) {
+        if (storageType == long.class) {
+          return Expressions.call(BuiltInMethod.TIMESTAMP_TO_LONG.method, e);
+        }
+        if (storageType == Long.class) {
+          return Expressions.call(BuiltInMethod.TIMESTAMP_TO_LONG_OPTIONAL.method, e);
+        }
+      }
+    }
+    return e;
+  }
 }
 
 // End EnumUtils.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRelImplementor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRelImplementor.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRelImplementor.java
index 1c446a3..bd77f2b 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRelImplementor.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRelImplementor.java
@@ -39,7 +39,6 @@ import org.apache.calcite.linq4j.tree.Types;
 import org.apache.calcite.linq4j.tree.Visitor;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.runtime.Bindable;
-import org.apache.calcite.runtime.Utilities;
 import org.apache.calcite.util.BuiltInMethod;
 
 import com.google.common.base.Function;
@@ -49,6 +48,7 @@ import com.google.common.collect.Iterables;
 import com.google.common.collect.Maps;
 
 import java.io.Serializable;
+import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
 import java.lang.reflect.ParameterizedType;
 import java.lang.reflect.Type;
@@ -239,9 +239,7 @@ public class EnumerableRelImplementor extends JavaRelImplementor {
               ? Expressions.equal(
                   Expressions.field(thisParameter, field.getName()),
                   Expressions.field(thatParameter, field.getName()))
-              : Expressions.call(
-                  Utilities.class,
-                  "equal",
+              : Expressions.call(BuiltInMethod.OBJECTS_EQUAL.method,
                   Expressions.field(thisParameter, field.getName()),
                   Expressions.field(thatParameter, field.getName())));
     }
@@ -270,13 +268,14 @@ public class EnumerableRelImplementor extends JavaRelImplementor {
     blockBuilder3.add(
         Expressions.declare(0, hParameter, constantZero));
     for (Types.RecordField field : type.getRecordFields()) {
+      final Method method = BuiltInMethod.HASH.method;
       blockBuilder3.add(
           Expressions.statement(
               Expressions.assign(
                   hParameter,
                   Expressions.call(
-                      Utilities.class,
-                      "hash",
+                      method.getDeclaringClass(),
+                      method.getName(),
                       ImmutableList.of(
                           hParameter,
                           Expressions.field(thisParameter, field))))));
@@ -312,9 +311,11 @@ public class EnumerableRelImplementor extends JavaRelImplementor {
     for (Types.RecordField field : type.getRecordFields()) {
       MethodCallExpression compareCall;
       try {
-        compareCall = Expressions.call(
-            Utilities.class,
-            field.nullable() ? "compareNullsLast" : "compare",
+        final Method method = (field.nullable()
+            ? BuiltInMethod.COMPARE_NULLS_LAST
+            : BuiltInMethod.COMPARE).method;
+        compareCall = Expressions.call(method.getDeclaringClass(),
+            method.getName(),
             Expressions.field(thisParameter, field),
             Expressions.field(thatParameter, field));
       } catch (RuntimeException e) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java
index af42c16..918833e 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java
@@ -40,6 +40,7 @@ import org.apache.calcite.util.Util;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
+import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
 import java.lang.reflect.Type;
 import java.util.AbstractList;
@@ -301,17 +302,17 @@ public class PhysTypeImpl implements PhysType {
       final boolean descending =
           collation.getDirection()
               == RelFieldCollation.Direction.DESCENDING;
+      final Method method = (fieldNullable(index)
+          ? (nullsFirst ^ descending
+              ? BuiltInMethod.COMPARE_NULLS_FIRST
+              : BuiltInMethod.COMPARE_NULLS_LAST)
+          : BuiltInMethod.COMPARE).method;
       body.add(
           Expressions.statement(
               Expressions.assign(
                   parameterC,
-                  Expressions.call(
-                      Utilities.class,
-                      fieldNullable(index)
-                          ? (nullsFirst ^ descending
-                              ? "compareNullsFirst"
-                              : "compareNullsLast")
-                          : "compare",
+                  Expressions.call(method.getDeclaringClass(),
+                      method.getName(),
                       arg0,
                       arg1))));
       body.add(

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/main/java/org/apache/calcite/adapter/enumerable/ReflectiveCallNotNullImplementor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/ReflectiveCallNotNullImplementor.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/ReflectiveCallNotNullImplementor.java
index 6414e03..ee2ad1f 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/ReflectiveCallNotNullImplementor.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/ReflectiveCallNotNullImplementor.java
@@ -16,16 +16,13 @@
  */
 package org.apache.calcite.adapter.enumerable;
 
-import org.apache.calcite.linq4j.tree.ConstantUntypedNull;
 import org.apache.calcite.linq4j.tree.Expression;
 import org.apache.calcite.linq4j.tree.Expressions;
 import org.apache.calcite.linq4j.tree.NewExpression;
 import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.runtime.SqlFunctions;
 
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
-import java.util.ArrayList;
 import java.util.List;
 
 /**
@@ -49,7 +46,8 @@ public class ReflectiveCallNotNullImplementor implements NotNullImplementor {
 
   public Expression implement(RexToLixTranslator translator,
       RexCall call, List<Expression> translatedOperands) {
-    translatedOperands = fromInternal(translatedOperands);
+    translatedOperands =
+        EnumUtils.fromInternal(method.getParameterTypes(), translatedOperands);
     if ((method.getModifiers() & Modifier.STATIC) != 0) {
       return Expressions.call(method, translatedOperands);
     } else {
@@ -61,36 +59,6 @@ public class ReflectiveCallNotNullImplementor implements NotNullImplementor {
     }
   }
 
-  protected List<Expression> fromInternal(List<Expression> expressions) {
-    final List<Expression> list = new ArrayList<>();
-    final Class[] types = method.getParameterTypes();
-    for (int i = 0; i < expressions.size(); i++) {
-      list.add(fromInternal(expressions.get(i), types[i]));
-    }
-    return list;
-  }
-
-  protected Expression fromInternal(Expression e, Class<?> targetType) {
-    if (e == ConstantUntypedNull.INSTANCE) {
-      return e;
-    }
-    if (!(e.getType() instanceof Class)) {
-      return e;
-    }
-    if (targetType.isAssignableFrom((Class) e.getType())) {
-      return e;
-    }
-    if (targetType == java.sql.Date.class) {
-      return Expressions.call(SqlFunctions.class, "internalToDate", e);
-    }
-    if (targetType == java.sql.Time.class) {
-      return Expressions.call(SqlFunctions.class, "internalToTime", e);
-    }
-    if (targetType == java.sql.Timestamp.class) {
-      return Expressions.call(SqlFunctions.class, "internalToTimestamp", e);
-    }
-    return e;
-  }
 }
 
 // End ReflectiveCallNotNullImplementor.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
index 32c043c..20872b6 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
@@ -1042,9 +1042,12 @@ public class RexImpTable {
       Expression acc = add.accumulator().get(0);
       Expression arg = add.arguments().get(0);
       SqlAggFunction aggregation = info.aggregation();
+      final Method method = (aggregation == MIN
+          ? BuiltInMethod.LESSER
+          : BuiltInMethod.GREATER).method;
       Expression next = Expressions.call(
-          SqlFunctions.class,
-          aggregation == MIN ? "lesser" : "greater",
+          method.getDeclaringClass(),
+          method.getName(),
           acc,
           Expressions.unbox(arg));
       accAdvance(add, acc, next);

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
index eed3d9d..2af6b46 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
@@ -181,7 +181,7 @@ public class RexToLixTranslator {
     List<Type> storageTypes = null;
     if (outputPhysType != null) {
       final RelDataType rowType = outputPhysType.getRowType();
-      storageTypes = new ArrayList<Type>(rowType.getFieldCount());
+      storageTypes = new ArrayList<>(rowType.getFieldCount());
       for (int i = 0; i < rowType.getFieldCount(); i++) {
         storageTypes.add(outputPhysType.getJavaFieldType(i));
       }
@@ -217,6 +217,7 @@ public class RexToLixTranslator {
   Expression translate(RexNode expr, RexImpTable.NullAs nullAs,
       Type storageType) {
     Expression expression = translate0(expr, nullAs, storageType);
+    expression = EnumUtils.enforce(storageType, expression);
     assert expression != null;
     return list.append("v", expression);
   }
@@ -500,9 +501,7 @@ public class RexToLixTranslator {
       }
       InputGetter getter =
           correlates.apply(((RexCorrelVariable) target).getName());
-      Expression res =
-          getter.field(list, fieldAccess.getField().getIndex(), storageType);
-      return res;
+      return getter.field(list, fieldAccess.getField().getIndex(), storageType);
     default:
       if (expr instanceof RexCall) {
         return translateCall((RexCall) expr, nullAs);
@@ -643,9 +642,17 @@ public class RexToLixTranslator {
   public List<Expression> translateList(
       List<RexNode> operandList,
       RexImpTable.NullAs nullAs) {
-    final List<Expression> list = new ArrayList<Expression>();
-    for (RexNode rex : operandList) {
-      list.add(translate(rex, nullAs));
+    return translateList(operandList, nullAs,
+        EnumUtils.internalTypes(operandList));
+  }
+
+  public List<Expression> translateList(
+      List<RexNode> operandList,
+      RexImpTable.NullAs nullAs,
+      List<? extends Type> storageTypes) {
+    final List<Expression> list = new ArrayList<>();
+    for (Pair<RexNode, ? extends Type> e : Pair.zip(operandList, storageTypes)) {
+      list.add(translate(e.left, nullAs, e.right));
     }
     return list;
   }
@@ -663,7 +670,7 @@ public class RexToLixTranslator {
    * @return translated expressions
    */
   public List<Expression> translateList(List<? extends RexNode> operandList) {
-    return translateList(operandList, null);
+    return translateList(operandList, EnumUtils.internalTypes(operandList));
   }
 
   /**
@@ -682,7 +689,7 @@ public class RexToLixTranslator {
    */
   public List<Expression> translateList(List<? extends RexNode> operandList,
       List<? extends Type> storageTypes) {
-    final List<Expression> list = new ArrayList<Expression>(operandList.size());
+    final List<Expression> list = new ArrayList<>(operandList.size());
 
     for (int i = 0; i < operandList.size(); i++) {
       RexNode rex = operandList.get(i);
@@ -812,6 +819,19 @@ public class RexToLixTranslator {
         }
       }
       return Expressions.box(operand, toBox);
+    } else if (toType == java.sql.Date.class) {
+      // E.g. from "int" or "Integer" to "java.sql.Date",
+      // generate "SqlFunctions.internalToDate".
+      return Expressions.call(BuiltInMethod.INTERNAL_TO_DATE.method, operand);
+    } else if (toType == java.sql.Time.class) {
+      // E.g. from "int" or "Integer" to "java.sql.Time",
+      // generate "SqlFunctions.internalToTime".
+      return Expressions.call(BuiltInMethod.INTERNAL_TO_TIME.method, operand);
+    } else if (toType == java.sql.Timestamp.class) {
+      // E.g. from "long" or "Long" to "java.sql.Timestamp",
+      // generate "SqlFunctions.internalToTimestamp".
+      return Expressions.call(BuiltInMethod.INTERNAL_TO_TIMESTAMP.method,
+          operand);
     } else if (toType == BigDecimal.class) {
       if (fromBox != null) {
         // E.g. from "Integer" to "BigDecimal".
@@ -1048,6 +1068,7 @@ public class RexToLixTranslator {
    * it is not null. It is easier to throw (and caller will always handle)
    * than to check exhaustively beforehand. */
   static class AlwaysNull extends ControlFlowException {
+    @SuppressWarnings("ThrowableInstanceNeverThrown")
     public static final AlwaysNull INSTANCE = new AlwaysNull();
 
     private AlwaysNull() {}

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/main/java/org/apache/calcite/adapter/enumerable/StrictAggImplementor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/StrictAggImplementor.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/StrictAggImplementor.java
index 8b463e8..a34813c 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/StrictAggImplementor.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/StrictAggImplementor.java
@@ -65,13 +65,7 @@ public abstract class StrictAggImplementor implements AggImplementor {
     if (!needTrackEmptySet) {
       return subState;
     }
-    boolean hasNullableArgs = false;
-    for (RelDataType type : info.parameterRelTypes()) {
-      if (type.isNullable()) {
-        hasNullableArgs = true;
-        break;
-      }
-    }
+    final boolean hasNullableArgs = anyNullable(info.parameterRelTypes());
     trackNullsPerRow = !(info instanceof WinAggContext) || hasNullableArgs;
 
     List<Type> res = new ArrayList<>(subState.size() + 1);
@@ -80,8 +74,20 @@ public abstract class StrictAggImplementor implements AggImplementor {
     return res;
   }
 
+  private boolean anyNullable(List<? extends RelDataType> types) {
+    for (RelDataType type : types) {
+      if (type.isNullable()) {
+        return true;
+      }
+    }
+    return false;
+  }
+
   public List<Type> getNotNullState(AggContext info) {
-    return Collections.singletonList(Primitive.unbox(info.returnType()));
+    Type type = info.returnType();
+    type = EnumUtils.fromInternal(type);
+    type = Primitive.unbox(type);
+    return Collections.singletonList(type);
   }
 
   public final void implementReset(AggContext info, AggResetContext reset) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java b/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
index e3211ea..68b297e 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
@@ -27,18 +27,19 @@ import org.apache.calcite.schema.AggregateFunction;
 import org.apache.calcite.schema.Function;
 import org.apache.calcite.schema.FunctionParameter;
 import org.apache.calcite.schema.ScalarFunction;
-import org.apache.calcite.schema.Schemas;
 import org.apache.calcite.schema.Table;
 import org.apache.calcite.schema.TableFunction;
 import org.apache.calcite.schema.TableMacro;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlOperatorBinding;
 import org.apache.calcite.sql.SqlSyntax;
 import org.apache.calcite.sql.type.FamilyOperandTypeChecker;
 import org.apache.calcite.sql.type.InferTypes;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlReturnTypeInference;
 import org.apache.calcite.sql.type.SqlTypeFamily;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.validate.SqlMoniker;
@@ -263,14 +264,11 @@ public class CalciteCatalogReader implements Prepare.CatalogReader {
         OperandTypes.family(typeFamilies, optional);
     final List<RelDataType> paramTypes = toSql(argTypes);
     if (function instanceof ScalarFunction) {
-      return new SqlUserDefinedFunction(name,
-          ReturnTypes.explicit(Schemas.proto((ScalarFunction) function)),
+      return new SqlUserDefinedFunction(name, infer((ScalarFunction) function),
           InferTypes.explicit(argTypes), typeChecker, paramTypes, function);
     } else if (function instanceof AggregateFunction) {
-      final RelDataType returnType =
-          ((AggregateFunction) function).getReturnType(typeFactory);
       return new SqlUserDefinedAggFunction(name,
-          ReturnTypes.explicit(returnType), InferTypes.explicit(argTypes),
+          infer((AggregateFunction) function), InferTypes.explicit(argTypes),
           typeChecker, (AggregateFunction) function);
     } else if (function instanceof TableMacro) {
       return new SqlUserDefinedTableMacro(name, ReturnTypes.CURSOR,
@@ -285,21 +283,43 @@ public class CalciteCatalogReader implements Prepare.CatalogReader {
     }
   }
 
+  private SqlReturnTypeInference infer(final ScalarFunction function) {
+    return new SqlReturnTypeInference() {
+      public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
+        final RelDataType type = function.getReturnType(typeFactory);
+        return toSql(type);
+      }
+    };
+  }
+
+  private SqlReturnTypeInference infer(final AggregateFunction function) {
+    return new SqlReturnTypeInference() {
+      public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
+        final RelDataType type = function.getReturnType(typeFactory);
+        return toSql(type);
+      }
+    };
+  }
+
   private List<RelDataType> toSql(List<RelDataType> types) {
     return Lists.transform(types,
         new com.google.common.base.Function<RelDataType, RelDataType>() {
-          public RelDataType apply(RelDataType input) {
-            if (input instanceof RelDataTypeFactoryImpl.JavaType
-                && ((RelDataTypeFactoryImpl.JavaType) input).getJavaClass()
-                == Object.class) {
-              return typeFactory.createTypeWithNullability(
-                  typeFactory.createSqlType(SqlTypeName.ANY), true);
-            }
-            return typeFactory.toSql(input);
+          public RelDataType apply(RelDataType type) {
+            return toSql(type);
           }
         });
   }
 
+  private RelDataType toSql(RelDataType type) {
+    if (type instanceof RelDataTypeFactoryImpl.JavaType
+        && ((RelDataTypeFactoryImpl.JavaType) type).getJavaClass()
+        == Object.class) {
+      return typeFactory.createTypeWithNullability(
+          typeFactory.createSqlType(SqlTypeName.ANY), true);
+    }
+    return typeFactory.toSql(type);
+  }
+
   public List<SqlOperator> getOperatorList() {
     return null;
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/main/java/org/apache/calcite/runtime/FlatLists.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/FlatLists.java b/core/src/main/java/org/apache/calcite/runtime/FlatLists.java
index 4a001cf..f3cfae4 100644
--- a/core/src/main/java/org/apache/calcite/runtime/FlatLists.java
+++ b/core/src/main/java/org/apache/calcite/runtime/FlatLists.java
@@ -23,6 +23,7 @@ import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 import java.util.ListIterator;
+import java.util.Objects;
 import java.util.RandomAccess;
 
 /**
@@ -261,8 +262,8 @@ public class FlatLists {
       }
       if (o instanceof Flat2List) {
         Flat2List that = (Flat2List) o;
-        return Utilities.equal(this.t0, that.t0)
-            && Utilities.equal(this.t1, that.t1);
+        return Objects.equals(this.t0, that.t0)
+            && Objects.equals(this.t1, that.t1);
       }
       return Arrays.asList(t0, t1).equals(o);
     }
@@ -386,9 +387,9 @@ public class FlatLists {
       }
       if (o instanceof Flat3List) {
         Flat3List that = (Flat3List) o;
-        return Utilities.equal(this.t0, that.t0)
-            && Utilities.equal(this.t1, that.t1)
-            && Utilities.equal(this.t2, that.t2);
+        return Objects.equals(this.t0, that.t0)
+            && Objects.equals(this.t1, that.t1)
+            && Objects.equals(this.t2, that.t2);
       }
       return o.equals(this);
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java b/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
index 22b0aaa..8abdb97 100644
--- a/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
+++ b/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
@@ -1040,6 +1040,10 @@ public class SqlFunctions {
         : (Short) cannotConvert(o, short.class);
   }
 
+  /** Converts the Java type used for UDF parameters of SQL DATE type
+   * ({@link java.sql.Date}) to internal representation (int).
+   *
+   * <p>Converse of {@link #internalToDate(int)}. */
   public static int toInt(java.util.Date v) {
     return toInt(v, LOCAL_TZ);
   }
@@ -1062,6 +1066,10 @@ public class SqlFunctions {
     return toLong(v, LOCAL_TZ);
   }
 
+  /** Converts the Java type used for UDF parameters of SQL TIME type
+   * ({@link java.sql.Time}) to internal representation (int).
+   *
+   * <p>Converse of {@link #internalToTime(int)}. */
   public static int toInt(java.sql.Time v) {
     return (int) (toLong(v) % DateTimeUtils.MILLIS_PER_DAY);
   }
@@ -1085,6 +1093,10 @@ public class SqlFunctions {
         : (Integer) cannotConvert(o, int.class);
   }
 
+  /** Converts the Java type used for UDF parameters of SQL TIMESTAMP type
+   * ({@link java.sql.Timestamp}) to internal representation (long).
+   *
+   * <p>Converse of {@link #internalToTimestamp(long)}. */
   public static long toLong(Timestamp v) {
     return toLong(v, LOCAL_TZ);
   }
@@ -1175,33 +1187,34 @@ public class SqlFunctions {
 
   /** Converts the internal representation of a SQL DATE (int) to the Java
    * type used for UDF parameters ({@link java.sql.Date}). */
-  public static java.sql.Date internalToDate(int x) {
-    return new java.sql.Date(x * DateTimeUtils.MILLIS_PER_DAY);
+  public static java.sql.Date internalToDate(int v) {
+    final long t = v * DateTimeUtils.MILLIS_PER_DAY;
+    return new java.sql.Date(t - LOCAL_TZ.getOffset(t));
   }
 
   /** As {@link #internalToDate(int)} but allows nulls. */
-  public static java.sql.Date internalToDate(Integer x) {
-    return x == null ? null : internalToDate(x.intValue());
+  public static java.sql.Date internalToDate(Integer v) {
+    return v == null ? null : internalToDate(v.intValue());
   }
 
   /** Converts the internal representation of a SQL TIME (int) to the Java
    * type used for UDF parameters ({@link java.sql.Time}). */
-  public static java.sql.Time internalToTime(int x) {
-    return new java.sql.Time(x);
+  public static java.sql.Time internalToTime(int v) {
+    return new java.sql.Time(v - LOCAL_TZ.getOffset(v));
   }
 
-  public static java.sql.Time internalToTime(Integer x) {
-    return x == null ? null : internalToTime(x.intValue());
+  public static java.sql.Time internalToTime(Integer v) {
+    return v == null ? null : internalToTime(v.intValue());
   }
 
   /** Converts the internal representation of a SQL TIMESTAMP (long) to the Java
    * type used for UDF parameters ({@link java.sql.Timestamp}). */
-  public static java.sql.Timestamp internalToTimestamp(long x) {
-    return new java.sql.Timestamp(x);
+  public static java.sql.Timestamp internalToTimestamp(long v) {
+    return new java.sql.Timestamp(v - LOCAL_TZ.getOffset(v));
   }
 
-  public static java.sql.Timestamp internalToTimestamp(Long x) {
-    return x == null ? null : internalToTimestamp(x.longValue());
+  public static java.sql.Timestamp internalToTimestamp(Long v) {
+    return v == null ? null : internalToTimestamp(v.longValue());
   }
 
   // Don't need shortValueOf etc. - Short.valueOf is sufficient.

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/main/java/org/apache/calcite/runtime/Utilities.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/Utilities.java b/core/src/main/java/org/apache/calcite/runtime/Utilities.java
index 691afd5..cf4fc0c 100644
--- a/core/src/main/java/org/apache/calcite/runtime/Utilities.java
+++ b/core/src/main/java/org/apache/calcite/runtime/Utilities.java
@@ -29,6 +29,8 @@ public class Utilities {
   protected Utilities() {
   }
 
+  /** @deprecated Use {@link java.util.Objects#equals}. */
+  @Deprecated // to be removed before 2.0
   public static boolean equal(Object o0, Object o1) {
     // Same as java.lang.Objects.equals (JDK 1.7 and later)
     // and com.google.common.base.Objects.equal

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlUtil.java b/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
index 66fb911..87e73e0 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
@@ -493,7 +493,7 @@ public abstract class SqlUtil {
         final RelDataType argType = p.right;
         final RelDataType paramType = p.left;
         if (argType != null
-            && !SqlTypeUtil.canAssignFrom(paramType, argType)) {
+            && !SqlTypeUtil.canCastFrom(paramType, argType, false)) {
           iter.remove();
           continue loop;
         }

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/main/java/org/apache/calcite/sql/type/SqlTypeExplicitPrecedenceList.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeExplicitPrecedenceList.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeExplicitPrecedenceList.java
index 1da123d..4fe315b 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeExplicitPrecedenceList.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeExplicitPrecedenceList.java
@@ -71,6 +71,7 @@ public class SqlTypeExplicitPrecedenceList
       ImmutableMap.<SqlTypeName, SqlTypeExplicitPrecedenceList>builder()
           .put(SqlTypeName.BOOLEAN, list(SqlTypeName.BOOLEAN))
           .put(SqlTypeName.TINYINT, numeric(SqlTypeName.TINYINT))
+          .put(SqlTypeName.SMALLINT, numeric(SqlTypeName.SMALLINT))
           .put(SqlTypeName.INTEGER, numeric(SqlTypeName.INTEGER))
           .put(SqlTypeName.BIGINT, numeric(SqlTypeName.BIGINT))
           .put(SqlTypeName.DECIMAL, numeric(SqlTypeName.DECIMAL))

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
index 658acd4..204b7ea 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
@@ -589,6 +589,7 @@ public abstract class SqlTypeUtil {
    * @return true if type has a representation as a Java primitive (ignoring
    * nullability)
    */
+  @Deprecated // to be removed before 2.0
   public static boolean isJavaPrimitive(RelDataType type) {
     SqlTypeName typeName = type.getSqlTypeName();
     if (typeName == null) {
@@ -614,6 +615,7 @@ public abstract class SqlTypeUtil {
   /**
    * @return class name of the wrapper for the primitive data type.
    */
+  @Deprecated // to be removed before 2.0
   public static String getPrimitiveWrapperJavaClassName(RelDataType type) {
     if (type == null) {
       return null;
@@ -627,6 +629,7 @@ public abstract class SqlTypeUtil {
     case BOOLEAN:
       return "Boolean";
     default:
+      //noinspection deprecation
       return getNumericJavaClassName(type);
     }
   }
@@ -634,6 +637,7 @@ public abstract class SqlTypeUtil {
   /**
    * @return class name of the numeric data type.
    */
+  @Deprecated // to be removed before 2.0
   public static String getNumericJavaClassName(RelDataType type) {
     if (type == null) {
       return null;

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
index 4c7b41c..c899cb8 100644
--- a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
+++ b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
@@ -56,6 +56,7 @@ import org.apache.calcite.runtime.FlatLists;
 import org.apache.calcite.runtime.ResultSetEnumerable;
 import org.apache.calcite.runtime.SortedMultiMap;
 import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.runtime.Utilities;
 import org.apache.calcite.schema.FilterableTable;
 import org.apache.calcite.schema.ModifiableTable;
 import org.apache.calcite.schema.ProjectableFilterableTable;
@@ -82,6 +83,7 @@ import java.util.Comparator;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 import java.util.TimeZone;
 import javax.sql.DataSource;
 
@@ -254,10 +256,15 @@ public enum BuiltInMethod {
   IS_TRUE(SqlFunctions.class, "isTrue", Boolean.class),
   IS_NOT_FALSE(SqlFunctions.class, "isNotFalse", Boolean.class),
   NOT(SqlFunctions.class, "not", Boolean.class),
+  LESSER(SqlFunctions.class, "lesser", Comparable.class, Comparable.class),
+  GREATER(SqlFunctions.class, "greater", Comparable.class, Comparable.class),
   MODIFIABLE_TABLE_GET_MODIFIABLE_COLLECTION(ModifiableTable.class,
       "getModifiableCollection"),
   SCANNABLE_TABLE_SCAN(ScannableTable.class, "scan", DataContext.class),
   STRING_TO_BOOLEAN(SqlFunctions.class, "toBoolean", String.class),
+  INTERNAL_TO_DATE(SqlFunctions.class, "internalToDate", int.class),
+  INTERNAL_TO_TIME(SqlFunctions.class, "internalToTime", int.class),
+  INTERNAL_TO_TIMESTAMP(SqlFunctions.class, "internalToTimestamp", long.class),
   STRING_TO_DATE(DateTimeUtils.class, "dateStringToUnixDate", String.class),
   STRING_TO_TIME(DateTimeUtils.class, "timeStringToUnixDate", String.class),
   STRING_TO_TIMESTAMP(DateTimeUtils.class, "timestampStringToUnixDate",
@@ -288,8 +295,13 @@ public enum BuiltInMethod {
   BOOLEAN_TO_STRING(SqlFunctions.class, "toString", boolean.class),
   JDBC_ARRAY_TO_LIST(SqlFunctions.class, "arrayToList", java.sql.Array.class),
   OBJECT_TO_STRING(Object.class, "toString"),
-  OBJECTS_EQUAL(com.google.common.base.Objects.class, "equal", Object.class,
-      Object.class),
+  OBJECTS_EQUAL(Objects.class, "equals", Object.class, Object.class),
+  HASH(Utilities.class, "hash", int.class, Object.class),
+  COMPARE(Utilities.class, "compare", Comparable.class, Comparable.class),
+  COMPARE_NULLS_FIRST(Utilities.class, "compareNullsFirst", Comparable.class,
+      Comparable.class),
+  COMPARE_NULLS_LAST(Utilities.class, "compareNullsLast", Comparable.class,
+      Comparable.class),
   ROUND_LONG(SqlFunctions.class, "round", long.class, long.class),
   ROUND_INT(SqlFunctions.class, "round", int.class, int.class),
   DATE_TO_INT(SqlFunctions.class, "toInt", java.util.Date.class),

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/CalciteAssert.java b/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
index f9e2bea..4b36b51 100644
--- a/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
+++ b/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
@@ -275,7 +275,8 @@ public class CalciteAssert {
             throw new AssertionError("expected 1 column");
           }
           final String resultString = resultSet.getString(1);
-          assertEquals(expected, Util.toLinux(resultString));
+          assertEquals(expected,
+              resultString == null ? null : Util.toLinux(resultString));
           return null;
         } catch (SQLException e) {
           throw new RuntimeException(e);

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/test/java/org/apache/calcite/test/UdfTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/UdfTest.java b/core/src/test/java/org/apache/calcite/test/UdfTest.java
index 7d707fe..a8d7766 100644
--- a/core/src/test/java/org/apache/calcite/test/UdfTest.java
+++ b/core/src/test/java/org/apache/calcite/test/UdfTest.java
@@ -105,6 +105,12 @@ public class UdfTest {
         + "         {\n"
         + "           name: 'COUNT_ARGS',\n"
         + "           className: '"
+        + Smalls.CountArgs1NullableFunction.class.getName()
+        + "'\n"
+        + "         },\n"
+        + "         {\n"
+        + "           name: 'COUNT_ARGS',\n"
+        + "           className: '"
         + Smalls.CountArgs2Function.class.getName()
         + "'\n"
         + "         },\n"
@@ -261,6 +267,14 @@ public class UdfTest {
         .returns("P0=0; P1=1; P2=2\n");
   }
 
+  @Test public void testUdfOverloadedNullable() {
+    final CalciteAssert.AssertThat with = withUdf();
+    with.query("values (\"adhoc\".count_args(),\n"
+        + " \"adhoc\".count_args(cast(null as smallint)),\n"
+        + " \"adhoc\".count_args(0, 0))")
+        .returns("EXPR$0=0; EXPR$1=-1; EXPR$2=2\n");
+  }
+
   /** Tests passing parameters to user-defined function by name. */
   @Test public void testUdfArgumentName() {
     final CalciteAssert.AssertThat with = withUdf();
@@ -586,6 +600,54 @@ public class UdfTest {
     with.query("values \"adhoc\".\"timestampFun\"(cast(null as timestamp))")
         .returnsValue("-1");
   }
+
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1041">[CALCITE-1041]
+   * User-defined function returns DATE or TIMESTAMP value</a>. */
+  @Test public void testReturnDate() {
+    final CalciteAssert.AssertThat with = withUdf();
+    with.query("values \"adhoc\".\"toDateFun\"(0)")
+        .returnsValue("1970-01-01");
+    with.query("values \"adhoc\".\"toDateFun\"(1)")
+        .returnsValue("1970-01-02");
+    with.query("values \"adhoc\".\"toDateFun\"(cast(null as bigint))")
+        .returnsValue(null);
+    with.query("values \"adhoc\".\"toTimeFun\"(0)")
+        .returnsValue("00:00:00");
+    with.query("values \"adhoc\".\"toTimeFun\"(90000)")
+        .returnsValue("00:01:30");
+    with.query("values \"adhoc\".\"toTimeFun\"(cast(null as bigint))")
+        .returnsValue(null);
+    with.query("values \"adhoc\".\"toTimestampFun\"(0)")
+        .returnsValue("1970-01-01 00:00:00");
+    with.query("values \"adhoc\".\"toTimestampFun\"(86490000)")
+        .returnsValue("1970-01-02 00:01:30");
+    with.query("values \"adhoc\".\"toTimestampFun\"(cast(null as bigint))")
+        .returnsValue(null);
+  }
+
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1041">[CALCITE-1041]
+   * User-defined function returns DATE or TIMESTAMP value</a>. */
+  @Test public void testReturnDate2() {
+    final CalciteAssert.AssertThat with = withUdf();
+    with.query("select * from (values 0) as t(c)\n"
+        + "where \"adhoc\".\"toTimestampFun\"(c) in (\n"
+        + "  cast('1970-01-01 00:00:00' as timestamp),\n"
+        + "  cast('1997-02-01 00:00:00' as timestamp))")
+        .returnsValue("0");
+    with.query("select * from (values 0) as t(c)\n"
+        + "where \"adhoc\".\"toTimestampFun\"(c) in (\n"
+        + "  timestamp '1970-01-01 00:00:00',\n"
+        + "  timestamp '1997-02-01 00:00:00')")
+        .returnsValue("0");
+    with.query("select * from (values 0) as t(c)\n"
+        + "where \"adhoc\".\"toTimestampFun\"(c) in (\n"
+        + "  '1970-01-01 00:00:00',\n"
+        + "  '1997-02-01 00:00:00')")
+        .returnsValue("0");
+  }
+
 }
 
 // End UdfTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/test/java/org/apache/calcite/util/Smalls.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/util/Smalls.java b/core/src/test/java/org/apache/calcite/util/Smalls.java
index 62cc064..d9e07ef 100644
--- a/core/src/test/java/org/apache/calcite/util/Smalls.java
+++ b/core/src/test/java/org/apache/calcite/util/Smalls.java
@@ -33,6 +33,7 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelProtoDataType;
 import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.runtime.SqlFunctions;
 import org.apache.calcite.schema.QueryableTable;
 import org.apache.calcite.schema.ScannableTable;
 import org.apache.calcite.schema.SchemaPlus;
@@ -418,20 +419,34 @@ public class Smalls {
   public static class AllTypesFunction {
     private AllTypesFunction() {}
 
-    public static long dateFun(java.sql.Date x) { return x == null ? -1L : x.getTime(); }
-    public static long timestampFun(java.sql.Timestamp x) { return x == null ? -1L : x.getTime(); }
-    public static long timeFun(java.sql.Time x) { return x == null ? -1L : x.getTime(); }
+    // We use SqlFunctions.toLong(Date) ratter than Date.getTime(),
+    // and SqlFunctions.internalToTimestamp(long) rather than new Date(long),
+    // because the contract of JDBC (also used by UDFs) is to represent
+    // date-time values in the LOCAL time zone.
 
-    public static java.sql.Date toDateFun(int x) { return new java.sql.Date(x); }
+    public static long dateFun(java.sql.Date v) {
+      return v == null ? -1L : SqlFunctions.toLong(v);
+    }
+    public static long timestampFun(java.sql.Timestamp v) {
+      return v == null ? -1L : SqlFunctions.toLong(v);
+    }
+    public static long timeFun(java.sql.Time v) {
+      return v == null ? -1L : SqlFunctions.toLong(v);
+    }
+
+    /** Overloaded, in a challenging way, with {@link #toDateFun(Long)}. */
+    public static java.sql.Date toDateFun(int v) {
+      return SqlFunctions.internalToDate(v);
+    }
 
-    public static java.sql.Date toDateFun(Long x) {
-      return x == null ? null : new java.sql.Date(x);
+    public static java.sql.Date toDateFun(Long v) {
+      return v == null ? null : SqlFunctions.internalToDate(v.intValue());
     }
-    public static java.sql.Timestamp toTimestampFun(Long x) {
-      return x == null ? null : new java.sql.Timestamp(x);
+    public static java.sql.Timestamp toTimestampFun(Long v) {
+      return SqlFunctions.internalToTimestamp(v);
     }
-    public static java.sql.Time toTimeFun(Long x) {
-      return x == null ? null : new java.sql.Time(x);
+    public static java.sql.Time toTimeFun(Long v) {
+      return v == null ? null : SqlFunctions.internalToTime(v.intValue());
     }
   }