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 2017/01/28 16:56:37 UTC

[2/4] calcite git commit: [CALCITE-1606] Add datetime scalar functions (Laurent Goujon)

[CALCITE-1606] Add datetime scalar functions (Laurent Goujon)

Add support for the missing JDBC/ODBC datetime scalar functions: YEAR,
MONTH, WEEK, DAYOFYEAR, DAYOFMONTH, DAYOFWEEK, HOUR, MINUTE, SECOND.
Also EXTRACT with the corresponding time-unit arguments.

Close apache/calcite#364


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

Branch: refs/heads/master
Commit: 94cb577898e0cab2c1acc92a981133323918660c
Parents: 7605d42
Author: Laurent Goujon <la...@dremio.com>
Authored: Thu Jan 26 13:41:50 2017 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri Jan 27 23:56:54 2017 -0800

----------------------------------------------------------------------
 core/src/main/codegen/templates/Parser.jj       |   5 +
 .../apache/calcite/sql/SqlJdbcFunctionCall.java |  10 ++
 .../calcite/sql/fun/SqlDatePartFunction.java    |  81 +++++++++
 .../calcite/sql/fun/SqlQuarterFunction.java     |  63 -------
 .../calcite/sql/fun/SqlStdOperatorTable.java    |  85 +++++++++-
 .../sql2rel/StandardConvertletTable.java        |  32 ----
 .../apache/calcite/sql/test/SqlAdvisorTest.java |   7 +-
 .../calcite/sql/test/SqlOperatorBaseTest.java   | 170 ++++++++++++++++---
 site/_docs/reference.md                         |  28 ++-
 9 files changed, 348 insertions(+), 133 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/94cb5778/core/src/main/codegen/templates/Parser.jj
----------------------------------------------------------------------
diff --git a/core/src/main/codegen/templates/Parser.jj b/core/src/main/codegen/templates/Parser.jj
index 6887a21..dcc0558 100644
--- a/core/src/main/codegen/templates/Parser.jj
+++ b/core/src/main/codegen/templates/Parser.jj
@@ -4738,6 +4738,7 @@ SqlIdentifier ReservedFunctionName() :
         | <FLOOR>
         | <FUSION>
         | <GROUPING>
+        | <HOUR>
         | <LAST_VALUE>
         | <LN>
         | <LOCALTIME>
@@ -4745,7 +4746,9 @@ SqlIdentifier ReservedFunctionName() :
         | <LOWER>
         | <MAX>
         | <MIN>
+        | <MINUTE>
         | <MOD>
+        | <MONTH>
         | <NULLIF>
         | <OCTET_LENGTH>
         | <PERCENT_RANK>
@@ -4754,6 +4757,7 @@ SqlIdentifier ReservedFunctionName() :
         | <REGR_SXX>
         | <REGR_SYY>
         | <ROW_NUMBER>
+        | <SECOND>
         | <SQRT>
         | <STDDEV_POP>
         | <STDDEV_SAMP>
@@ -4762,6 +4766,7 @@ SqlIdentifier ReservedFunctionName() :
         | <USER>
         | <VAR_POP>
         | <VAR_SAMP>
+        | <YEAR>
     )
     {
         return new SqlIdentifier(unquotedIdentifier(), getPos());

http://git-wip-us.apache.org/repos/asf/calcite/blob/94cb5778/core/src/main/java/org/apache/calcite/sql/SqlJdbcFunctionCall.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlJdbcFunctionCall.java b/core/src/main/java/org/apache/calcite/sql/SqlJdbcFunctionCall.java
index 34c42be..20a01da 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlJdbcFunctionCall.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlJdbcFunctionCall.java
@@ -707,7 +707,17 @@ public class SqlJdbcFunctionCall extends SqlFunction {
                   operands[0]);
             }
           });
+      map.put("YEAR", simple(SqlStdOperatorTable.YEAR));
       map.put("QUARTER", simple(SqlStdOperatorTable.QUARTER));
+      map.put("MONTH", simple(SqlStdOperatorTable.MONTH));
+      map.put("WEEK", simple(SqlStdOperatorTable.WEEK));
+      map.put("DAYOFYEAR", simple(SqlStdOperatorTable.DAYOFYEAR));
+      map.put("DAYOFMONTH", simple(SqlStdOperatorTable.DAYOFMONTH));
+      map.put("DAYOFWEEK", simple(SqlStdOperatorTable.DAYOFWEEK));
+      map.put("HOUR", simple(SqlStdOperatorTable.HOUR));
+      map.put("MINUTE", simple(SqlStdOperatorTable.MINUTE));
+      map.put("SECOND", simple(SqlStdOperatorTable.SECOND));
+
       map.put("RTRIM",
           new SimpleMakeCall(SqlStdOperatorTable.TRIM) {
             @Override public SqlCall createCall(SqlParserPos pos,

http://git-wip-us.apache.org/repos/asf/calcite/blob/94cb5778/core/src/main/java/org/apache/calcite/sql/fun/SqlDatePartFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlDatePartFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlDatePartFunction.java
new file mode 100644
index 0000000..e6c5bde
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlDatePartFunction.java
@@ -0,0 +1,81 @@
+/*
+ * 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.avatica.util.TimeUnit;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperandCountRange;
+import org.apache.calcite.sql.parser.SqlParserPos;
+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.SqlOperandCountRanges;
+import org.apache.calcite.sql.validate.SqlValidator;
+
+import java.util.List;
+
+/**
+ * SqlDatePartFunction represents the SQL:1999 standard {@code YEAR},
+ * {@code QUARTER}, {@code MONTH} and {@code DAY} functions.
+ */
+public class SqlDatePartFunction extends SqlFunction {
+  //~ Constructors -----------------------------------------------------------
+  private final TimeUnit timeUnit;
+
+  public SqlDatePartFunction(String name, TimeUnit timeUnit) {
+    super(name,
+        SqlKind.OTHER,
+        ReturnTypes.BIGINT_NULLABLE,
+        InferTypes.FIRST_KNOWN,
+        OperandTypes.DATETIME,
+        SqlFunctionCategory.TIMEDATE);
+    this.timeUnit = timeUnit;
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  @Override public SqlNode rewriteCall(SqlValidator validator, SqlCall call) {
+    final List<SqlNode> operands = call.getOperandList();
+    final SqlParserPos pos = call.getParserPosition();
+    return SqlStdOperatorTable.EXTRACT.createCall(pos,
+        new SqlIntervalQualifier(timeUnit, null, SqlParserPos.ZERO),
+        operands.get(0));
+  }
+
+  public SqlOperandCountRange getOperandCountRange() {
+    return SqlOperandCountRanges.of(1);
+  }
+
+  public String getSignatureTemplate(int operandsCount) {
+    assert 1 == operandsCount;
+    return "{0}({1})";
+  }
+
+  public boolean checkOperandTypes(SqlCallBinding callBinding,
+      boolean throwOnFailure) {
+    return OperandTypes.DATETIME.checkSingleOperandType(callBinding,
+        callBinding.operand(0), 0, throwOnFailure);
+  }
+}
+
+// End SqlDatePartFunction.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/94cb5778/core/src/main/java/org/apache/calcite/sql/fun/SqlQuarterFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlQuarterFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlQuarterFunction.java
deleted file mode 100644
index b76c1dd..0000000
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlQuarterFunction.java
+++ /dev/null
@@ -1,63 +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.SqlCallBinding;
-import org.apache.calcite.sql.SqlFunction;
-import org.apache.calcite.sql.SqlFunctionCategory;
-import org.apache.calcite.sql.SqlKind;
-import org.apache.calcite.sql.SqlOperandCountRange;
-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.SqlOperandCountRanges;
-
-/**
- * SqlQuarterFunction represents the SQL:1999 standard {@code QUARTER}
- * function. Determines Quarter (1,2,3,4) of a given date.
- */
-public class SqlQuarterFunction extends SqlFunction {
-  //~ Constructors -----------------------------------------------------------
-
-  public SqlQuarterFunction() {
-    super("QUARTER",
-        SqlKind.OTHER,
-        ReturnTypes.BIGINT_NULLABLE,
-        InferTypes.FIRST_KNOWN,
-        OperandTypes.DATETIME,
-        SqlFunctionCategory.TIMEDATE);
-  }
-
-  //~ Methods ----------------------------------------------------------------
-
-  public SqlOperandCountRange getOperandCountRange() {
-    return SqlOperandCountRanges.of(1);
-  }
-
-  public String getSignatureTemplate(int operandsCount) {
-    assert 1 == operandsCount;
-    return "{0}({1})";
-  }
-
-  public boolean checkOperandTypes(SqlCallBinding callBinding,
-      boolean throwOnFailure) {
-    return OperandTypes.DATETIME.checkSingleOperandType(callBinding,
-        callBinding.operand(0), 0, throwOnFailure);
-  }
-}
-
-// End SqlQuarterFunction.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/94cb5778/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 5c71006..dfbdfa0 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,6 +16,7 @@
  */
 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;
@@ -1534,12 +1535,94 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
   public static final SqlFunction EXTRACT = new SqlExtractFunction();
 
   /**
+   * The SQL <code>YEAR</code> operator. Returns the Year
+   * from a DATETIME  E.g.<br>
+   * <code>YEAR(date '2008-9-23')</code> returns <code>
+   * 2008</code>
+   */
+  public static final SqlDatePartFunction YEAR =
+      new SqlDatePartFunction("YEAR", TimeUnit.YEAR);
+
+  /**
    * The SQL <code>QUARTER</code> operator. Returns the Quarter
    * from a DATETIME  E.g.<br>
    * <code>QUARTER(date '2008-9-23')</code> returns <code>
    * 3</code>
    */
-  public static final SqlQuarterFunction QUARTER = new SqlQuarterFunction();
+  public static final SqlDatePartFunction QUARTER =
+      new SqlDatePartFunction("QUARTER", TimeUnit.QUARTER);
+
+  /**
+   * The SQL <code>MONTH</code> operator. Returns the Month
+   * from a DATETIME  E.g.<br>
+   * <code>MONTH(date '2008-9-23')</code> returns <code>
+   * 9</code>
+   */
+  public static final SqlDatePartFunction MONTH =
+      new SqlDatePartFunction("MONTH", TimeUnit.MONTH);
+
+  /**
+   * The SQL <code>WEEK</code> operator. Returns the Week
+   * from a DATETIME  E.g.<br>
+   * <code>WEEK(date '2008-9-23')</code> returns <code>
+   * 39</code>
+   */
+  public static final SqlDatePartFunction WEEK =
+      new SqlDatePartFunction("WEEK", TimeUnit.WEEK);
+
+  /**
+   * The SQL <code>DAYOFYEAR</code> operator. Returns the DOY
+   * from a DATETIME  E.g.<br>
+   * <code>DAYOFYEAR(date '2008-9-23')</code> returns <code>
+   * 267</code>
+   */
+  public static final SqlDatePartFunction DAYOFYEAR =
+      new SqlDatePartFunction("DAYOFYEAR", TimeUnit.DOY);
+
+  /**
+   * The SQL <code>DAYOFMONTH</code> operator. Returns the Day
+   * from a DATETIME  E.g.<br>
+   * <code>DAYOFMONTH(date '2008-9-23')</code> returns <code>
+   * 23</code>
+   */
+  public static final SqlDatePartFunction DAYOFMONTH =
+      new SqlDatePartFunction("DAYOFMONTH", TimeUnit.DAY);
+
+  /**
+   * The SQL <code>DAYOFWEEK</code> operator. Returns the DOW
+   * from a DATETIME  E.g.<br>
+   * <code>DAYOFWEEK(date '2008-9-23')</code> returns <code>
+   * 2</code>
+   */
+  public static final SqlDatePartFunction DAYOFWEEK =
+      new SqlDatePartFunction("DAYOFWEEK", TimeUnit.DOW);
+
+  /**
+   * The SQL <code>HOUR</code> operator. Returns the Hour
+   * from a DATETIME  E.g.<br>
+   * <code>HOUR(timestamp '2008-9-23 01:23:45')</code> returns <code>
+   * 1</code>
+   */
+  public static final SqlDatePartFunction HOUR =
+      new SqlDatePartFunction("HOUR", TimeUnit.HOUR);
+
+  /**
+   * The SQL <code>MINUTE</code> operator. Returns the Minute
+   * from a DATETIME  E.g.<br>
+   * <code>MINUTE(timestamp '2008-9-23 01:23:45')</code> returns <code>
+   * 23</code>
+   */
+  public static final SqlDatePartFunction MINUTE =
+      new SqlDatePartFunction("MINUTE", TimeUnit.MINUTE);
+
+  /**
+   * The SQL <code>SECOND</code> operator. Returns the Second
+   * from a DATETIME  E.g.<br>
+   * <code>SECOND(timestamp '2008-9-23 01:23:45')</code> returns <code>
+   * 45</code>
+   */
+  public static final SqlDatePartFunction SECOND =
+      new SqlDatePartFunction("SECOND", TimeUnit.SECOND);
 
   /**
    * The ELEMENT operator, used to convert a multiset with only one item to a

http://git-wip-us.apache.org/repos/asf/calcite/blob/94cb5778/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java b/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
index f1c2e1e..40fecf7 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
@@ -60,7 +60,6 @@ import org.apache.calcite.sql.fun.SqlMapValueConstructor;
 import org.apache.calcite.sql.fun.SqlMultisetQueryConstructor;
 import org.apache.calcite.sql.fun.SqlMultisetValueConstructor;
 import org.apache.calcite.sql.fun.SqlOverlapsOperator;
-import org.apache.calcite.sql.fun.SqlQuarterFunction;
 import org.apache.calcite.sql.fun.SqlRowOperator;
 import org.apache.calcite.sql.fun.SqlSequenceValueOperator;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
@@ -744,37 +743,6 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
     return res;
   }
 
-  /**
-   * Converts a call to the {@code QUARTER} function.
-   *
-   * <p>Called automatically via reflection.
-   */
-  public RexNode convertQuarter(
-      SqlRexContext cx,
-      SqlQuarterFunction op,
-      SqlCall call) {
-    final List<SqlNode> operands = call.getOperandList();
-    assert operands.size() == 1;
-    RexNode x = cx.convertExpression(operands.get(0));
-    final RexBuilder rexBuilder = cx.getRexBuilder();
-    final RelDataTypeFactory typeFactory = cx.getTypeFactory();
-    final RelDataType resType =
-        typeFactory.createTypeWithNullability(
-            typeFactory.createSqlType(SqlTypeName.BIGINT),
-            x.getType().isNullable());
-    RexNode res =
-        rexBuilder.makeCall(
-            resType,
-            SqlStdOperatorTable.EXTRACT_DATE,
-            ImmutableList.of(rexBuilder.makeFlag(TimeUnitRange.MONTH), x));
-    res = rexBuilder.makeCall(SqlStdOperatorTable.MINUS, res,
-        rexBuilder.makeExactLiteral(BigDecimal.ONE));
-    res = divide(rexBuilder, res, TimeUnit.QUARTER.multiplier);
-    res = rexBuilder.makeCall(SqlStdOperatorTable.PLUS, res,
-        rexBuilder.makeExactLiteral(BigDecimal.ONE));
-    return res;
-  }
-
   private static BigDecimal getFactor(TimeUnit unit) {
     switch (unit) {
     case DAY:

http://git-wip-us.apache.org/repos/asf/calcite/blob/94cb5778/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java b/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
index 24a4626..b318b7c 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
@@ -149,6 +149,7 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
           "KEYWORD(FLOOR)",
           "KEYWORD(FUSION)",
           "KEYWORD(GROUPING)",
+          "KEYWORD(HOUR)",
           "KEYWORD(INTERVAL)",
           "KEYWORD(LAST_VALUE)",
           "KEYWORD(LN)",
@@ -157,7 +158,9 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
           "KEYWORD(LOWER)",
           "KEYWORD(MAX)",
           "KEYWORD(MIN)",
+          "KEYWORD(MINUTE)",
           "KEYWORD(MOD)",
+          "KEYWORD(MONTH)",
           "KEYWORD(MULTISET)",
           "KEYWORD(NEW)",
           "KEYWORD(NEXT)",
@@ -174,6 +177,7 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
           "KEYWORD(REGR_SYY)",
           "KEYWORD(ROW)",
           "KEYWORD(ROW_NUMBER)",
+          "KEYWORD(SECOND)",
           "KEYWORD(SESSION_USER)",
           "KEYWORD(SPECIFIC)",
           "KEYWORD(SQRT)",
@@ -191,7 +195,8 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
           "KEYWORD(UPPER)",
           "KEYWORD(USER)",
           "KEYWORD(VAR_POP)",
-          "KEYWORD(VAR_SAMP)");
+          "KEYWORD(VAR_SAMP)",
+          "KEYWORD(YEAR)");
 
   protected static final List<String> SELECT_KEYWORDS =
       Arrays.asList(

http://git-wip-us.apache.org/repos/asf/calcite/blob/94cb5778/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 24c33f5..76aab95 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
@@ -1739,45 +1739,37 @@ public abstract class SqlOperatorBaseTest {
     if (false) {
       tester.checkScalar("{fn DAYNAME(date)}", null, "");
     }
-    if (false) {
-      tester.checkScalar("{fn DAYOFMONTH(date)}", null, "");
-    }
-    if (false) {
-      tester.checkScalar("{fn DAYOFWEEK(date)}", null, "");
-    }
-    if (false) {
-      tester.checkScalar("{fn DAYOFYEAR(date)}", null, "");
-    }
-    if (false) {
-      tester.checkScalar("{fn HOUR(time)}", null, "");
-    }
-    if (false) {
-      tester.checkScalar("{fn MINUTE(time)}", null, "");
-    }
-    if (false) {
-      tester.checkScalar("{fn MONTH(date)}", null, "");
-    }
+    tester.checkScalar("{fn DAYOFMONTH(DATE '2014-12-10')}", 10,
+        "BIGINT NOT NULL");
+    tester.checkFails("{fn DAYOFWEEK(DATE '2014-12-10')}",
+        "cannot translate call EXTRACT.*",
+        true);
+    tester.checkFails("{fn DAYOFYEAR(DATE '2014-12-10')}",
+        "cannot translate call EXTRACT.*",
+        true);
+    tester.checkScalar("{fn HOUR(TIMESTAMP '2014-12-10 12:34:56')}", 12,
+        "BIGINT NOT NULL");
+    tester.checkScalar("{fn MINUTE(TIMESTAMP '2014-12-10 12:34:56')}", 34,
+        "BIGINT NOT NULL");
+    tester.checkScalar("{fn MONTH(DATE '2014-12-10')}", 12, "BIGINT NOT NULL");
     if (false) {
       tester.checkScalar("{fn MONTHNAME(date)}", null, "");
     }
     tester.checkType("{fn NOW()}", "TIMESTAMP(0) NOT NULL");
     tester.checkScalar("{fn QUARTER(DATE '2014-12-10')}", "4",
         "BIGINT NOT NULL");
-    if (false) {
-      tester.checkScalar("{fn SECOND(time)}", null, "");
-    }
+    tester.checkScalar("{fn SECOND(TIMESTAMP '2014-12-10 12:34:56')}", 56,
+        "BIGINT NOT NULL");
     tester.checkScalar("{fn TIMESTAMPADD(HOUR, 5,"
         + " TIMESTAMP '2014-03-29 12:34:56')}",
         "2014-03-29 17:34:56", "TIMESTAMP(0) NOT NULL");
     tester.checkScalar("{fn TIMESTAMPDIFF(HOUR,"
         + " TIMESTAMP '2014-03-29 12:34:56',"
         + " TIMESTAMP '2014-03-29 12:34:56')}", "0", "INTEGER NOT NULL");
-    if (false) {
-      tester.checkScalar("{fn WEEK(date)}", null, "");
-    }
-    if (false) {
-      tester.checkScalar("{fn YEAR(date)}", null, "");
-    }
+    tester.checkFails("{fn WEEK(DATE '2014-12-10')}",
+        "cannot translate call EXTRACT.*",
+        true);
+    tester.checkScalar("{fn YEAR(DATE '2014-12-10')}", 2014, "BIGINT NOT NULL");
 
     // System Functions
     tester.checkType("{fn DATABASE()}", "VARCHAR(2000) NOT NULL");
@@ -4750,6 +4742,19 @@ public abstract class SqlOperatorBaseTest {
     tester.setFor(SqlStdOperatorTable.FUSION, VM_FENNEL, VM_JAVA);
   }
 
+  @Test public void testYear() {
+    tester.setFor(
+        SqlStdOperatorTable.YEAR,
+        VM_FENNEL,
+        VM_JAVA);
+
+    tester.checkScalar(
+        "year(date '2008-1-23')",
+        "2008",
+        "BIGINT NOT NULL");
+    tester.checkNull("year(cast(null as date))");
+  }
+
   @Test public void testQuarter() {
     tester.setFor(
         SqlStdOperatorTable.QUARTER,
@@ -4807,6 +4812,117 @@ public abstract class SqlOperatorBaseTest {
     tester.checkNull("quarter(cast(null as date))");
   }
 
+  @Test public void testMonth() {
+    tester.setFor(
+        SqlStdOperatorTable.MONTH,
+        VM_FENNEL,
+        VM_JAVA);
+
+    tester.checkScalar(
+        "month(date '2008-1-23')",
+        "1",
+        "BIGINT NOT NULL");
+    tester.checkNull("month(cast(null as date))");
+  }
+
+  @Test public void testWeek() {
+    tester.setFor(
+        SqlStdOperatorTable.WEEK,
+        VM_FENNEL,
+        VM_JAVA);
+    // TODO: Not implemented in operator test execution code
+    tester.checkFails(
+        "week(date '2008-1-23')",
+        "cannot translate call EXTRACT.*",
+        true);
+    tester.checkFails(
+        "week(cast(null as date))",
+        "cannot translate call EXTRACT.*",
+        true);
+  }
+
+  @Test public void testDayOfYear() {
+    tester.setFor(
+        SqlStdOperatorTable.DAYOFYEAR,
+        VM_FENNEL,
+        VM_JAVA);
+    // TODO: Not implemented in operator test execution code
+    tester.checkFails(
+        "dayofyear(date '2008-1-23')",
+        "cannot translate call EXTRACT.*",
+        true);
+    tester.checkFails(
+        "dayofyear(cast(null as date))",
+        "cannot translate call EXTRACT.*",
+        true);
+  }
+
+  @Test public void testDayOfMonth() {
+    tester.setFor(
+        SqlStdOperatorTable.DAYOFMONTH,
+        VM_FENNEL,
+        VM_JAVA);
+    tester.checkScalar(
+        "dayofmonth(date '2008-1-23')",
+        "23",
+        "BIGINT NOT NULL");
+    tester.checkNull("dayofmonth(cast(null as date))");
+  }
+
+  @Test public void testDayOfWeek() {
+    tester.setFor(
+        SqlStdOperatorTable.DAYOFWEEK,
+        VM_FENNEL,
+        VM_JAVA);
+    // TODO: Not implemented in operator test execution code
+    tester.checkFails(
+        "dayofweek(date '2008-1-23')",
+        "cannot translate call EXTRACT.*",
+        true);
+    tester.checkFails("dayofweek(cast(null as date))",
+        "cannot translate call EXTRACT.*",
+        true);
+  }
+
+  @Test public void testHour() {
+    tester.setFor(
+        SqlStdOperatorTable.HOUR,
+        VM_FENNEL,
+        VM_JAVA);
+
+    tester.checkScalar(
+        "hour(timestamp '2008-1-23 12:34:56')",
+        "12",
+        "BIGINT NOT NULL");
+    tester.checkNull("hour(cast(null as timestamp))");
+  }
+
+  @Test public void testMinute() {
+    tester.setFor(
+        SqlStdOperatorTable.MINUTE,
+        VM_FENNEL,
+        VM_JAVA);
+
+    tester.checkScalar(
+        "minute(timestamp '2008-1-23 12:34:56')",
+        "34",
+        "BIGINT NOT NULL");
+    tester.checkNull("minute(cast(null as timestamp))");
+  }
+
+  @Test public void testSecond() {
+    tester.setFor(
+        SqlStdOperatorTable.SECOND,
+        VM_FENNEL,
+        VM_JAVA);
+
+    tester.checkScalar(
+        "second(timestamp '2008-1-23 12:34:56')",
+        "56",
+        "BIGINT NOT NULL");
+    tester.checkNull("second(cast(null as timestamp))");
+  }
+
   @Test public void testExtractIntervalYearMonth() {
     tester.setFor(
         SqlStdOperatorTable.EXTRACT,

http://git-wip-us.apache.org/repos/asf/calcite/blob/94cb5778/site/_docs/reference.md
----------------------------------------------------------------------
diff --git a/site/_docs/reference.md b/site/_docs/reference.md
index 3b9c3d3..37f33d9 100644
--- a/site/_docs/reference.md
+++ b/site/_docs/reference.md
@@ -1026,6 +1026,16 @@ Not implemented:
 | EXTRACT(timeUnit FROM datetime) | Extracts and returns the value of a specified datetime field from a datetime value expression
 | FLOOR(datetime TO timeUnit) | Rounds *datetime* down to *timeUnit*
 | CEIL(datetime TO timeUnit) | Rounds *datetime* up to *timeUnit*
+| YEAR(date)                | Equivalent to `EXTRACT(YEAR FROM date)`. Returns an integer.
+| QUARTER(date)             | Equivalent to `EXTRACT(QUARTER FROM date)`. Returns an integer between 1 and 4.
+| MONTH(date)               | Equivalent to `EXTRACT(MONTH FROM date)`. Returns an integer between 1 and 12.
+| WEEK(date)                | Equivalent to `EXTRACT(WEEK FROM date)`. Returns an integer between 1 and 53.
+| DAYOFYEAR(date)           | Equivalent to `EXTRACT(DOY FROM date)`. Returns an integer between 1 and 366.
+| DAYOFMONTH(date)          | Equivalent to `EXTRACT(DAY FROM date)`. Returns an integer between 1 and 31.
+| DAYOFWEEK(date)           | Equivalent to `EXTRACT(DOW FROM date)`. Returns an integer between 1 and 7.
+| HOUR(date)                | Equivalent to `EXTRACT(HOUR FROM date)`. Returns an integer between 0 and 23.
+| MINUTE(date)              | Equivalent to `EXTRACT(MINUTE FROM date)`. Returns an integer between 0 and 59.
+| SECOND(date)              | Equivalent to `EXTRACT(SECOND FROM date)`. Returns an integer between 0 and 59.
 
 Not implemented:
 
@@ -1152,23 +1162,23 @@ Not implemented:
 | {fn CURDATE()}  | Equivalent to `CURRENT_DATE`
 | {fn CURTIME()}  | Equivalent to `LOCALTIME`
 | {fn NOW()}      | Equivalent to `LOCALTIMESTAMP`
+| {fn YEAR(date)} | Equivalent to `EXTRACT(YEAR FROM date)`. Returns an integer.
 | {fn QUARTER(date)} | Equivalent to `EXTRACT(QUARTER FROM date)`. Returns an integer between 1 and 4.
+| {fn MONTH(date)} | Equivalent to `EXTRACT(MONTH FROM date)`. Returns an integer between 1 and 12.
+| {fn WEEK(date)} | Equivalent to `EXTRACT(WEEK FROM date)`. Returns an integer between 1 and 53.
+| {fn DAYOFYEAR(date)} | Equivalent to `EXTRACT(DOY FROM date)`. Returns an integer between 1 and 366.
+| {fn DAYOFMONTH(date)} | Equivalent to `EXTRACT(DAY FROM date)`. Returns an integer between 1 and 31.
+| {fn DAYOFWEEK(date)} | Equivalent to `EXTRACT(DOW FROM date)`. Returns an integer between 1 and 7.
+| {fn HOUR(date)} | Equivalent to `EXTRACT(HOUR FROM date)`. Returns an integer between 0 and 23.
+| {fn MINUTE(date)} | Equivalent to `EXTRACT(MINUTE FROM date)`. Returns an integer between 0 and 59.
+| {fn SECOND(date)} | Equivalent to `EXTRACT(SECOND FROM date)`. Returns an integer between 0 and 59.
 | {fn TIMESTAMPADD(timeUnit, count, timestamp)} | Adds an interval of *count* *timeUnit*s to a timestamp
 | {fn TIMESTAMPDIFF(timeUnit, timestamp1, timestamp2)} | Subtracts *timestamp1* from *timestamp2* and returns the result in *timeUnit*s
 
 Not implemented:
 
 * {fn DAYNAME(date)}
-* {fn DAYOFMONTH(date)}
-* {fn DAYOFWEEK(date)}
-* {fn DAYOFYEAR(date)}
-* {fn HOUR(time)}
-* {fn MINUTE(time)}
-* {fn MONTH(date)}
 * {fn MONTHNAME(date)}
-* {fn SECOND(time)}
-* {fn WEEK(date)}
-* {fn YEAR(date)}
 
 #### System