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/12/22 09:59:20 UTC

[2/2] calcite git commit: [CALCITE-1552] Add RAND function, returning DOUBLE values in the range 0..1

[CALCITE-1552] Add RAND function, returning DOUBLE values in the range 0..1

Also, add seed to RAND_INTEGER. This means adding the random number
generator as state in the function object.

Rename SqlRandInteger to SqlRandIntegerFunction.


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

Branch: refs/heads/master
Commit: 6f761d367feb1819664f145b9f7732bff5e9c602
Parents: 599ea33
Author: Julian Hyde <jh...@apache.org>
Authored: Tue Dec 20 16:56:52 2016 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Thu Dec 22 01:12:52 2016 -0800

----------------------------------------------------------------------
 .../calcite/adapter/enumerable/RexImpTable.java | 38 +++++++++-
 .../apache/calcite/runtime/RandomFunction.java  | 74 ++++++++++++++++++
 .../apache/calcite/runtime/SqlFunctions.java    |  8 --
 .../apache/calcite/sql/fun/SqlRandFunction.java | 59 +++++++++++++++
 .../apache/calcite/sql/fun/SqlRandInteger.java  | 54 -------------
 .../calcite/sql/fun/SqlRandIntegerFunction.java | 59 +++++++++++++++
 .../calcite/sql/fun/SqlStdOperatorTable.java    | 80 ++++++++++----------
 .../org/apache/calcite/util/BuiltInMethod.java  |  7 +-
 .../calcite/sql/test/SqlOperatorBaseTest.java   | 23 +++++-
 core/src/test/resources/sql/misc.iq             | 80 ++++++++++++++++++++
 site/_docs/reference.md                         |  3 +-
 11 files changed, 380 insertions(+), 105 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/6f761d36/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 8b55c22..01c740c 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
@@ -157,6 +157,7 @@ import static org.apache.calcite.sql.fun.SqlStdOperatorTable.OVERLAY;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.PLUS;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.POSITION;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.POWER;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.RAND;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.RAND_INTEGER;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.RANK;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.REINTERPRET;
@@ -245,8 +246,34 @@ public class RexImpTable {
     defineMethod(LN, "ln", NullPolicy.STRICT);
     defineMethod(LOG10, "log10", NullPolicy.STRICT);
     defineMethod(ABS, "abs", NullPolicy.STRICT);
-    defineMethod(RAND_INTEGER, BuiltInMethod.RAND_INTEGER.method,
-        NullPolicy.STRICT);
+
+    defineImplementor(RAND, NullPolicy.STRICT,
+        new NotNullImplementor() {
+          final NotNullImplementor[] implementors = {
+            new ReflectiveCallNotNullImplementor(BuiltInMethod.RAND.method),
+            new ReflectiveCallNotNullImplementor(BuiltInMethod.RAND_SEED.method)
+          };
+          public Expression implement(RexToLixTranslator translator,
+              RexCall call, List<Expression> translatedOperands) {
+            return implementors[call.getOperands().size()]
+                .implement(translator, call, translatedOperands);
+          }
+        }, false);
+    defineImplementor(RAND_INTEGER, NullPolicy.STRICT,
+        new NotNullImplementor() {
+          final NotNullImplementor[] implementors = {
+            null,
+            new ReflectiveCallNotNullImplementor(
+                BuiltInMethod.RAND_INTEGER.method),
+            new ReflectiveCallNotNullImplementor(
+                BuiltInMethod.RAND_INTEGER_SEED.method)
+          };
+          public Expression implement(RexToLixTranslator translator,
+              RexCall call, List<Expression> translatedOperands) {
+            return implementors[call.getOperands().size()]
+                .implement(translator, call, translatedOperands);
+          }
+        }, false);
 
     // datetime
     defineImplementor(DATETIME_PLUS, NullPolicy.STRICT,
@@ -569,6 +596,13 @@ public class RexImpTable {
         operator, nullPolicy, new MethodImplementor(method), false);
   }
 
+  private void defineMethodReflective(
+      SqlOperator operator, Method method, NullPolicy nullPolicy) {
+    defineImplementor(
+        operator, nullPolicy, new ReflectiveCallNotNullImplementor(method),
+        false);
+  }
+
   private void defineUnary(
       SqlOperator operator, ExpressionType expressionType,
       NullPolicy nullPolicy) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/6f761d36/core/src/main/java/org/apache/calcite/runtime/RandomFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/RandomFunction.java b/core/src/main/java/org/apache/calcite/runtime/RandomFunction.java
new file mode 100644
index 0000000..e1a26c7
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/runtime/RandomFunction.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.runtime;
+
+import org.apache.calcite.linq4j.function.Deterministic;
+import org.apache.calcite.linq4j.function.Parameter;
+
+import java.util.Random;
+
+/**
+ * Function object for {@code RAND} and {@code RAND_INTEGER}, with and without
+ * seed.
+ */
+@SuppressWarnings("unused")
+public class RandomFunction {
+  private Random random;
+
+  /** Creates a RandomFunction.
+   *
+   * <p>Marked deterministic so that the code generator instantiates one once
+   * per query, not once per row. */
+  @Deterministic public RandomFunction() {
+  }
+
+  /** Implements the {@code RAND()} SQL function. */
+  public double rand() {
+    if (random == null) {
+      random = new Random();
+    }
+    return random.nextDouble();
+  }
+
+  /** Implements the {@code RAND(seed)} SQL function. */
+  public double randSeed(@Parameter(name = "seed") int seed) {
+    if (random == null) {
+      random = new Random(seed ^ (seed << 16));
+    }
+    return random.nextDouble();
+  }
+
+  /** Implements the {@code RAND_INTEGER(bound)} SQL function. */
+  public int randInteger(@Parameter(name = "bound") int bound) {
+    if (random == null) {
+      random = new Random();
+    }
+    return random.nextInt(bound);
+  }
+
+  /** Implements the {@code RAND_INTEGER(seed, bound)} SQL function. */
+  public int randIntegerSeed(@Parameter(name = "seed") int seed,
+      @Parameter(name = "bound") int bound) {
+    if (random == null) {
+      random = new Random(seed);
+    }
+    return random.nextInt(bound);
+  }
+
+}
+
+// End RandomFunction.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/6f761d36/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 2fd6586..87b5528 100644
--- a/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
+++ b/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
@@ -46,7 +46,6 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.Random;
 import java.util.TimeZone;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.regex.Pattern;
@@ -1053,13 +1052,6 @@ public class SqlFunctions {
     return b0.abs();
   }
 
-  // Random Function
-
-  @NonDeterministic
-  public static int randInteger(int n) {
-    return new Random().nextInt(n);
-  }
-
   // Helpers
 
   /** Helper for implementing MIN. Somewhat similar to LEAST operator. */

http://git-wip-us.apache.org/repos/asf/calcite/blob/6f761d36/core/src/main/java/org/apache/calcite/sql/fun/SqlRandFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlRandFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlRandFunction.java
new file mode 100644
index 0000000..08a3d0f
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlRandFunction.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.sql.fun;
+
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlSyntax;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+
+/**
+ * The <code>RAND</code> function. There are two overloads:
+ *
+ * <ul>
+ *   <li>RAND() returns a random double between 0 and 1
+ *   <li>RAND(seed) returns a random double between 0 and 1, initializing the
+ *   random number generator with seed on first call
+ * </ul>
+ */
+public class SqlRandFunction extends SqlFunction {
+  //~ Constructors -----------------------------------------------------------
+
+  public SqlRandFunction() {
+    super("RAND",
+        SqlKind.OTHER_FUNCTION,
+        ReturnTypes.DOUBLE,
+        null,
+        OperandTypes.or(OperandTypes.NILADIC, OperandTypes.NUMERIC),
+        SqlFunctionCategory.NUMERIC);
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  public SqlSyntax getSyntax() {
+    return SqlSyntax.FUNCTION;
+  }
+
+  // Plans referencing context variables should never be cached
+  public boolean isDynamicFunction() {
+    return true;
+  }
+}
+
+// End SqlRandFunction.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/6f761d36/core/src/main/java/org/apache/calcite/sql/fun/SqlRandInteger.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlRandInteger.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlRandInteger.java
deleted file mode 100644
index 82302af..0000000
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlRandInteger.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.calcite.sql.fun;
-
-import org.apache.calcite.sql.SqlFunction;
-import org.apache.calcite.sql.SqlFunctionCategory;
-import org.apache.calcite.sql.SqlKind;
-import org.apache.calcite.sql.SqlSyntax;
-import org.apache.calcite.sql.type.OperandTypes;
-import org.apache.calcite.sql.type.ReturnTypes;
-
-/**
- * The <code>RAND_INTEGER</code> function.
- */
-public class SqlRandInteger extends SqlFunction {
-  //~ Constructors -----------------------------------------------------------
-
-  public SqlRandInteger() {
-    super(
-        "RAND_INTEGER",
-        SqlKind.OTHER_FUNCTION,
-        ReturnTypes.INTEGER,
-        null,
-        OperandTypes.NUMERIC,
-        SqlFunctionCategory.NUMERIC);
-  }
-
-  //~ Methods ----------------------------------------------------------------
-
-  public SqlSyntax getSyntax() {
-    return SqlSyntax.FUNCTION;
-  }
-
-  // Plans referencing context variables should never be cached
-  public boolean isDynamicFunction() {
-    return true;
-  }
-}
-
-// End SqlRandInteger.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/6f761d36/core/src/main/java/org/apache/calcite/sql/fun/SqlRandIntegerFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlRandIntegerFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlRandIntegerFunction.java
new file mode 100644
index 0000000..af4bc79
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlRandIntegerFunction.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.sql.fun;
+
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlSyntax;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+
+/**
+ * The <code>RAND_INTEGER</code> function. There are two overloads:
+ *
+ * <ul>
+ *   <li>RAND_INTEGER(bound) returns a random integer between 0 and bound - 1
+ *   <li>RAND_INTEGER(seed, bound) returns a random integer between 0 and
+ *   bound - 1, initializing the random number generator with seed on first call
+ * </ul>
+ */
+public class SqlRandIntegerFunction extends SqlFunction {
+  //~ Constructors -----------------------------------------------------------
+
+  public SqlRandIntegerFunction() {
+    super("RAND_INTEGER",
+        SqlKind.OTHER_FUNCTION,
+        ReturnTypes.INTEGER,
+        null,
+        OperandTypes.or(OperandTypes.NUMERIC, OperandTypes.NUMERIC_NUMERIC),
+        SqlFunctionCategory.NUMERIC);
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  public SqlSyntax getSyntax() {
+    return SqlSyntax.FUNCTION;
+  }
+
+  // Plans referencing context variables should never be cached
+  public boolean isDynamicFunction() {
+    return true;
+  }
+}
+
+// End SqlRandIntegerFunction.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/6f761d36/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
index 1511909..4550dfc 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
@@ -16,7 +16,6 @@
  */
 package org.apache.calcite.sql.fun;
 
-import org.apache.calcite.avatica.util.TimeUnit;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.sql.SqlAggFunction;
@@ -51,7 +50,6 @@ import org.apache.calcite.sql.type.IntervalSqlType;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
 import org.apache.calcite.sql.type.SqlOperandCountRanges;
-import org.apache.calcite.sql.type.SqlReturnTypeInference;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.util.ReflectiveSqlOperatorTable;
 import org.apache.calcite.sql.validate.SqlModality;
@@ -229,10 +227,14 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
           InferTypes.FIRST_KNOWN,
           OperandTypes.DIVISION_OPERATOR);
 
-  /**
-   * Random Integer.
-   */
-  public static final SqlRandInteger RAND_INTEGER = new SqlRandInteger();
+  /** The {@code RAND_INTEGER([seed, ] bound)} function, which yields a random
+   * integer, optionally with seed. */
+  public static final SqlRandIntegerFunction RAND_INTEGER =
+      new SqlRandIntegerFunction();
+
+  /** The {@code RAND([seed])} function, which yields a random double,
+   * optionally with seed. */
+  public static final SqlRandFunction RAND = new SqlRandFunction();
 
   /**
    * Internal integer arithmetic division operator, '<code>/INT</code>'. This
@@ -459,22 +461,15 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
    * Infix datetime plus operator, '<code>DATETIME + INTERVAL</code>'.
    */
   public static final SqlSpecialOperator DATETIME_PLUS =
-      new SqlSpecialOperator(
-          "DATETIME_PLUS",
-          SqlKind.PLUS,
-          40,
-          true,
-          new SqlReturnTypeInference() {
-        @Override public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
+      new SqlSpecialOperator("DATETIME_PLUS", SqlKind.PLUS, 40, true, null,
+          InferTypes.FIRST_KNOWN, OperandTypes.PLUS_OPERATOR) {
+        @Override public RelDataType
+        inferReturnType(SqlOperatorBinding opBinding) {
           final RelDataTypeFactory typeFactory = opBinding.getTypeFactory();
-          final RelDataType unit = opBinding.getOperandType(1);
-          final TimeUnit addUnit;
-          if (unit instanceof IntervalSqlType) {
-            addUnit = unit.getIntervalQualifier().getStartUnit();
-          } else {
-            addUnit = null;
-          }
-          switch (addUnit) {
+          final RelDataType leftType = opBinding.getOperandType(0);
+          final IntervalSqlType unitType =
+              (IntervalSqlType) opBinding.getOperandType(1);
+          switch (unitType.getIntervalQualifier().getStartUnit()) {
           case HOUR:
           case MINUTE:
           case SECOND:
@@ -482,35 +477,44 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
           case MICROSECOND:
             return typeFactory.createTypeWithNullability(
                 typeFactory.createSqlType(SqlTypeName.TIMESTAMP),
-                opBinding.getOperandType(0).isNullable()
-                    || opBinding.getOperandType(1).isNullable());
+                leftType.isNullable() || unitType.isNullable());
           default:
-            return opBinding.getOperandType(0);
+            return leftType;
           }
         }
-      },
-          InferTypes.FIRST_KNOWN,
-          OperandTypes.PLUS_OPERATOR);
+      };
 
   /**
-   * Multiset MEMBER OF. Checks to see if a element belongs to a multiset.<br>
-   * Example:<br>
-   * <code>'green' MEMBER OF MULTISET['red','almost green','blue']</code>
-   * returns <code>false</code>.
+   * Multiset {@code MEMBER OF}, which returns whether a element belongs to a
+   * multiset.
+   *
+   * <p>For example, the following returns <code>false</code>:
+   *
+   * <blockquote>
+   * <code>'green' MEMBER OF MULTISET ['red','almost green','blue']</code>
+   * </blockquote>
    */
   public static final SqlBinaryOperator MEMBER_OF =
       new SqlMultisetMemberOfOperator();
 
   /**
    * Submultiset. Checks to see if an multiset is a sub-set of another
-   * multiset.<br>
-   * Example:<br>
-   * <code>MULTISET['green'] SUBMULTISET OF MULTISET['red','almost
-   * green','blue']</code> returns <code>false</code>.
+   * multiset.
+   *
+   * <p>For example, the following returns <code>false</code>:
    *
-   * <p>But <code>MULTISET['blue', 'red'] SUBMULTISET OF
-   * MULTISET['red','almost green','blue']</code> returns <code>true</code>
-   * (<b>NB</b> multisets is order independant)
+   * <blockquote>
+   * <code>MULTISET ['green'] SUBMULTISET OF
+   * MULTISET['red', 'almost green', 'blue']</code>
+   * </blockquote>
+   *
+   * <p>The following returns <code>true</code>, in part because multisets are
+   * order-independent:
+   *
+   * <blockquote>
+   * <code>MULTISET ['blue', 'red'] SUBMULTISET OF
+   * MULTISET ['red', 'almost green', 'blue']</code>
+   * </blockquote>
    */
   public static final SqlBinaryOperator SUBMULTISET_OF =
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/6f761d36/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 9968b86..4ac7fea 100644
--- a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
+++ b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
@@ -68,6 +68,7 @@ import org.apache.calcite.runtime.BinarySearch;
 import org.apache.calcite.runtime.Bindable;
 import org.apache.calcite.runtime.Enumerables;
 import org.apache.calcite.runtime.FlatLists;
+import org.apache.calcite.runtime.RandomFunction;
 import org.apache.calcite.runtime.ResultSetEnumerable;
 import org.apache.calcite.runtime.SortedMultiMap;
 import org.apache.calcite.runtime.SqlFunctions;
@@ -259,7 +260,11 @@ public enum BuiltInMethod {
   OVERLAY3(SqlFunctions.class, "overlay", String.class, String.class, int.class,
       int.class),
   POSITION(SqlFunctions.class, "position", String.class, String.class),
-  RAND_INTEGER(SqlFunctions.class, "randInteger", int.class),
+  RAND(RandomFunction.class, "rand"),
+  RAND_SEED(RandomFunction.class, "randSeed", int.class),
+  RAND_INTEGER(RandomFunction.class, "randInteger", int.class),
+  RAND_INTEGER_SEED(RandomFunction.class, "randIntegerSeed", int.class,
+      int.class),
   TRUNCATE(SqlFunctions.class, "truncate", String.class, int.class),
   TRUNCATE_OR_PAD(SqlFunctions.class, "truncateOrPad", String.class, int.class),
   TRIM(SqlFunctions.class, "trim", boolean.class, boolean.class, String.class,

http://git-wip-us.apache.org/repos/asf/calcite/blob/6f761d36/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java b/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
index 63ec1f9..3b90848 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
@@ -3809,7 +3809,22 @@ public abstract class SqlOperatorBaseTest {
     tester.checkNull("log10(cast(null as real))");
   }
 
-  @Test public void testRandomFunc() {
+  @Test public void testRandFunc() {
+    tester.setFor(SqlStdOperatorTable.RAND);
+    tester.checkFails("^rand^", "Column 'RAND' not found in any table", false);
+    for (int i = 0; i < 100; i++) {
+      // Result must always be between 0 and 1, inclusive.
+      tester.checkScalarApprox("rand()", "DOUBLE NOT NULL", 0.5, 0.5);
+    }
+  }
+
+  @Test public void testRandSeedFunc() {
+    tester.setFor(SqlStdOperatorTable.RAND);
+    tester.checkScalarApprox("rand(1)", "DOUBLE NOT NULL", 0.6016, 0.0001);
+    tester.checkScalarApprox("rand(2)", "DOUBLE NOT NULL", 0.4728, 0.0001);
+  }
+
+  @Test public void testRandIntegerFunc() {
     tester.setFor(SqlStdOperatorTable.RAND_INTEGER);
     for (int i = 0; i < 100; i++) {
       // Result must always be between 0 and 10, inclusive.
@@ -3818,6 +3833,12 @@ public abstract class SqlOperatorBaseTest {
     }
   }
 
+  @Test public void testRandIntegerSeedFunc() {
+    tester.setFor(SqlStdOperatorTable.RAND_INTEGER);
+    tester.checkScalar("rand_integer(1, 11)", 4, "INTEGER NOT NULL");
+    tester.checkScalar("rand_integer(2, 11)", 1, "INTEGER NOT NULL");
+  }
+
   @Test public void testAbsFunc() {
     tester.setFor(SqlStdOperatorTable.ABS);
     tester.checkScalarExact("abs(-1)", "1");

http://git-wip-us.apache.org/repos/asf/calcite/blob/6f761d36/core/src/test/resources/sql/misc.iq
----------------------------------------------------------------------
diff --git a/core/src/test/resources/sql/misc.iq b/core/src/test/resources/sql/misc.iq
index 841966c..eb4a6ce 100644
--- a/core/src/test/resources/sql/misc.iq
+++ b/core/src/test/resources/sql/misc.iq
@@ -1640,4 +1640,84 @@ values cast(' -5 ' as double);
 
 !ok
 
+# RAND_INTEGER with seed
+select i, rand_integer(1, 5) as r
+from (values 1, 2, 3, 4, 5) as t(i);
++---+---+
+| I | R |
++---+---+
+| 1 | 0 |
+| 2 | 3 |
+| 3 | 2 |
+| 4 | 3 |
+| 5 | 4 |
++---+---+
+(5 rows)
+
+!ok
+
+# Same query, should yield same results
+select i, rand_integer(1, 5) as r
+from (values 1, 2, 3, 4, 5) as t(i);
++---+---+
+| I | R |
++---+---+
+| 1 | 0 |
+| 2 | 3 |
+| 3 | 2 |
+| 4 | 3 |
+| 5 | 4 |
++---+---+
+(5 rows)
+
+!ok
+
+# Same query with different seed
+select i, rand_integer(-1, 5) as r
+from (values 1, 2, 3, 4, 5) as t(i);
++---+---+
+| I | R |
++---+---+
+| 1 | 3 |
+| 2 | 0 |
+| 3 | 4 |
+| 4 | 4 |
+| 5 | 4 |
++---+---+
+(5 rows)
+
+!ok
+
+# Same query with different bound
+select i, rand_integer(-1, 2) as r
+from (values 1, 2, 3, 4, 5) as t(i);
++---+---+
+| I | R |
++---+---+
+| 1 | 0 |
+| 2 | 0 |
+| 3 | 0 |
+| 4 | 1 |
+| 5 | 1 |
++---+---+
+(5 rows)
+
+!ok
+
+# RAND with seed
+select i, rand(-1) as r
+from (values 1, 2, 3, 4, 5) as t(i);
++---+---------------------+
+| I | R                   |
++---+---------------------+
+| 1 | 0.03305388522187047 |
+| 2 |  0.6573104025344794 |
+| 3 |  0.7450920948729041 |
+| 4 |  0.6624972807480889 |
+| 5 |  0.5532616835728703 |
++---+---------------------+
+(5 rows)
+
+!ok
+
 # End misc.iq

http://git-wip-us.apache.org/repos/asf/calcite/blob/6f761d36/site/_docs/reference.md
----------------------------------------------------------------------
diff --git a/site/_docs/reference.md b/site/_docs/reference.md
index 9a8c9bf..744be2e 100644
--- a/site/_docs/reference.md
+++ b/site/_docs/reference.md
@@ -963,7 +963,8 @@ The operator precedence and associativity, highest to lowest.
 | EXP(numeric)              | Returns *e* raised to the power of *numeric*
 | CEIL(numeric)             | Rounds *numeric* up, and returns the smallest number that is greater than or equal to *numeric*
 | FLOOR(numeric)            | Rounds *numeric* down, and returns the largest number that is less than or equal to *numeric*
-| RAND_INTEGER(numeric)     | Generates a random integer between 0 and *numeric* - 1 inclusive
+| RAND([seed])              | Generates a random double between 0 and 1 inclusive, optionally initializing the random number generator with *seed*
+| RAND_INTEGER([seed, ] numeric) | Generates a random integer between 0 and *numeric* - 1 inclusive, optionally initializing the random number generator with *seed*
 
 ### Character string operators and functions